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/08/24 17:52:12 UTC

[GitHub] [helix] zhangmeng916 opened a new pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

zhangmeng916 opened a new pull request #1307:
URL: https://github.com/apache/helix/pull/1307


   ### Issues
   
   - [X] My PR addresses the following Helix issues and references them in the PR description:
   
   Fixed #1306 
   
   ### Description
   
   - [X] Here are some details about my PR, including screenshots of any UI changes:
   
   This PR implements a cluster topology using a trie. The trie is constructed in a way that each non end node represents a domain, and each end node represents an instance. This data structure could help retrieve the topology under any given domain efficiently. 
   
   ### Tests
   
   - [X] The following tests are written for this issue:
   
   TestTrieClusterTopology.java
   
   - [ ] The following is the result of the "mvn test" command on the appropriate module:
   
   (Before CI test pass, please copy & paste the result of "mvn test")
   
   ### Documentation (Optional)
   
   - In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Commits
   
   - My commits all reference appropriate Apache Helix GitHub issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r485180072



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {

Review comment:
       This is the API by default gets the full topology of the cluster. The key of the map will be the first level of domain defined in topology, e.g. zone, faultdomain, etc. 




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-698111830


   > This PR is ready to merge. Approved by @jiajunwang
   > Final commit message:
   > Add TrieClusterTopology for retrieving hierarchical topology in a cluster.
   > 
   > * Add TrieNode class to define a node in the trie
   > * Add ClusterTrie class to handle the construction, validation and retrieval of nodes/paths in the trie.
   > * Add ClusterTopology class to provide different APIs for users to retrieve cluster topology information.
   > * Add APIs in HelixAdmin to retrieve ClusterTopology of a specific cluster.
   
   Test result, please.


----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r488313224



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();

Review comment:
       Please refer to the updated PR. We do not need to topology awareness to be enabled for API purpose.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493833267



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())
+            .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (topologyDef.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    return topologyDef;
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    TrieNode rootNode = new TrieNode("", "ROOT");
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < topologyKeys.length; i++) {
+        String key = topologyKeys[i] + CONNECTOR + entry.getValue().get(topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(path, topologyKeys[i]);

Review comment:
       Please see the above comments. Actually have the full path stored in each trie node help save a lot of efforts during query. We don't need to store parents, and don't need to compose the full path when return the map.




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493126494



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       It's both memory and additional computing for splitting/merging the path String.
   If we just put minimum info into trie, then there is no need to merge Strings. And there is no additional memory usage. Then when you read the tree, you don't need to split based on the ":" again. So I feel there would be a lot of saving on both the write and read side.
   
   To be specific, I think the node needs: 1. children map<domain value, TrieNode>, 2. domain Type (optional, since the recorded topology def list can be used to determine the type), 3. domain value (optional too, duplicate with the childrm map key, but could be convient).
   Then when you read, you can fill the type according to the depth of the tree and collect the domain value based on the children map key.
   The matter is to avoid String operations in which we create new String objects.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493828839



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       I understand your point. I tried a bit with removing path variable. The problem is that I will need to compose the path when I try to return the results. e.g. if the domain given is "zone":"0", I find the trie node and all its children, and when I return, I'll need to return "group:0/zone:0" as the full path, so I still need to build the string with parent information, there is no other way to know the domains above it. So the path is actually very convenient. What do you think?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r491122058



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       "start" means where we start to collect the children node information. E.g., if we have the input of (["group":"0"], ["zone":"1"]), then we know that we should get all the children information from node /group:0/zone:1, basically this is the start point. 
   And if we have an input as "zone" domainType, then we need to find all the starting nodes for that domainType, and then return all the children. e.g. they can be /group:0/zone:0, /group:0/zone:1, /group:1/zone:0, /group:1/zone:1 etc.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r494026155



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       Yep, I've moved them to clustertrie.




----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu removed a comment on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu removed a comment on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-679405672


   Minor correction here. 
   I think the enum Type mentioned above is used only when cluster level clusterConfig.Topology is not defined. Otherwise, we still parse the topology definition in cluster config.
   
   In function getClusterTopologySetting:
   ```
   private static ClusterTopologyConfig getClusterTopologySetting(ClusterConfig clusterConfig) {
       ClusterTopologyConfig clusterTopologyConfig = new ClusterTopologyConfig();
          ...
         if (topologyDef != null) {
           // We parse "/zone/rack/sub-rack/instance" into  ["zone", "rack", "sub-rack", "instance"].
         } else {
           // Use default cluster topology definition, i,e. /root/zone/instance
           clusterTopologyConfig.endNodeType = Types.INSTANCE.name();
           clusterTopologyConfig.faultZoneType = Types.ZONE.name();
         }
         ...
     }
   ```
   
   
   In my understanding, current Topology.java has similar functions like validateAndOrderDomain and the constructor  TrieClusterTopology. 
   Topology::getClusterTopologySetting returns a parsed list of cluster topology as the constructor here. 
   There are similar logic of validating the instance topology config aligns with cluster's config in computeInstanceTopologyMapHelper. 
   
   IMHO, I personally prefer reusing current code. Also I think its better to read from the tree since we already have the tree created for LB.
   
   Thanks,
   Xiaoyuan 


----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-679421854


   > getClusterTopologySetting
   
   The big problem I have using Topology is that the logic handling is different. If I'm going to use it, I'll need to introduce a lot of switch cases, as the main purpose for Topology is for rebalance instead of API response to users. Here're a few examples:
   1. if topology is null, it does
   clusterTopologyConfig.endNodeType = Types.INSTANCE.name();
   clusterTopologyConfig.faultZoneType = Types.ZONE.name();
   But I'll throw an exception and return.
   
   2. When an instance topology misses a value, it also fills the default value, but I will need to throw an exception. 
   
   3. When you find an instance topology is not defined, you'll throw an exception of terminating rebalance. This is not something I should have in the log and return to users.
   
   4. With current clusterTopologyConfig.topologyKeyDefaultValue's linkedhashmap, it is not as easy/efficient as a trie to return the topology under each domain. 
   
   In all, the logic of Topology is more specific for rebalancing usage. If I change it, it will become a complicated class that needs to adapt to different usage. I think the correct way in the future should be:
   Have a base class that returns the raw data in cluster config and instance config, and different classes extend it for different usage. 
   


----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476662851



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       Thanks for the detailed reply.
   
   I think the previous 3 points can be achieved by adding a flag and using non static 'DEFAULT_DOMAIN_PREFIX'. 
   For the point 4, 
   Please correct me if I am wrong, line 288 in constructTrie will also add a "default" node similar to 'clusterTopologyKeyDefaultValue'. Maybe we could read the tree and bypass default node. 
   
   
   In my own opinion, I think the question would be, do we construct topology tree in class Topology for these instances? If yes, its better to read from the existing tree. If not, I think it's ok to have another orthogonal data structure representing the hierarchy. 




----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476662851



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       Thanks for the detailed reply.
   
   I think the previous 3 points can be achieved by adding a flag to Topology::ClusterTopologyConfig and maybe using non static 'DEFAULT_DOMAIN_PREFIX'. 
   For the point 4, 
    Maybe we could read the tree and bypass default node (or throw exception when we find a default node in the path)
   
   
   In my own opinion, I think the question would be, do we construct topology tree in class Topology for these instances? If yes, its better to read from the existing tree. If not, I think it's ok to have another orthogonal data structure representing the hierarchy. 




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r490535956



##########
File path: helix-core/src/main/java/org/apache/helix/HelixAdmin.java
##########
@@ -420,6 +420,24 @@ void addStateModelDef(String clusterName, String stateModelDef, StateModelDefini
    */
   void removeCloudConfig(String clusterName);
 
+  /**
+   * Get the topology of a specific cluster
+   * @param clusterName
+   */
+  Map<String, List<String>> getAllTopology(String clusterName);
+
+  /**
+   * Get all the instances under the fault zone
+   * @param clusterName
+   */
+  Map<String, List<String>> getInstancesUnderFaultZone(String clusterName);
+
+  /**
+   * Get all the instances whose domain config is not valid
+   * @param clusterName
+   */
+  List<String> getInvalidInstances(String clusterName);
+

Review comment:
       Discussed offline, and we will only keep one API in HelixAdmin. Updated.




----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476662851



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       Thanks for the detailed reply.
   
   I think the previous 3 points can be achieved by adding a flag to Topology::ClusterTopologyConfig and maybe using non static 'DEFAULT_DOMAIN_PREFIX'. Just like how we accommodate the 'faultZone' compute in this class. 
   
   For the point 4, 
   Maybe we could read the tree and bypass default node (or throw exception when we find a default node in the path)
   
   
   In my own opinion, I think the final question would be, do we construct topology tree in class Topology for these instances? If yes, its better to read from the same tree. If not, I think it's ok to have another orthogonal data structure representing the hierarchy. 




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493070718



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,155 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef = Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream()

Review comment:
       I see what you mean. I trimmed spaces after split. I think space should be allowed. Updated PR.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r494026067



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,249 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, final ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(Collections.emptyMap());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domainMap A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domainMap) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domainMap);
+    TrieNode startNode = getNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domainMap) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domainMap == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domainMap.size(); i++) {
+      if (!domainMap.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domainMap.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getNode(LinkedHashMap<String, String> domainMap) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domainMap.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domainType) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getNodeKey().equals(domainType)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       Agree. Moved to cluster trie.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r489577867



##########
File path: helix-core/src/main/java/org/apache/helix/HelixAdmin.java
##########
@@ -420,6 +420,24 @@ void addStateModelDef(String clusterName, String stateModelDef, StateModelDefini
    */
   void removeCloudConfig(String clusterName);
 
+  /**
+   * Get the topology of a specific cluster
+   * @param clusterName
+   */
+  Map<String, List<String>> getAllTopology(String clusterName);
+
+  /**
+   * Get all the instances under the fault zone
+   * @param clusterName
+   */
+  Map<String, List<String>> getInstancesUnderFaultZone(String clusterName);
+
+  /**
+   * Get all the instances whose domain config is not valid
+   * @param clusterName
+   */
+  List<String> getInvalidInstances(String clusterName);
+

Review comment:
       If we return an object, users will still need to parse it by themselves. It's a lot of work if you look at the functions in ClusterTrie. I don't think we want to have this requirement on them. Or you mean users need to call the functions inside ClusterTrie? then they need to know all the available functions? Please let me know.




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493050479



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       I don't get the reason. With your example, it should be very simple to distinguish since these 3 zone:0 nodes are under different parent nodes. And while you are traversing, the algorithm can construct the full path based on parents' information. You can even keep a stack with paths for this purpose.




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493115546



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieNode.java
##########
@@ -0,0 +1,59 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class TrieNode {
+  // A mapping between trie key and children nodes.
+  private Map<String, TrieNode> _children;
+
+  // the complete path/prefix leading to the current node.
+  private final String _path;
+
+  private final String _nodeKey;
+
+  TrieNode(String path, String nodeKey) {
+    _path = path;
+    _nodeKey = nodeKey;
+    _children = new HashMap<>();
+  }
+
+  public Map<String, TrieNode> getChildren() {
+    return _children;
+  }
+
+  public String getPath() {
+    return _path;
+  }
+
+  public String getNodeKey() {
+    return _nodeKey;
+  }
+
+  public void addChildrenMap(Map <String, TrieNode> children) {

Review comment:
       I guess this is not necessary?

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())

Review comment:
       nit, the first tirm becomes unnecessary with the later one in the map().

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())
+            .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (topologyDef.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    return topologyDef;
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    TrieNode rootNode = new TrieNode("", "ROOT");
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";

Review comment:
       StringBuilder for the performance.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())
+            .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (topologyDef.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    return topologyDef;
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    TrieNode rootNode = new TrieNode("", "ROOT");
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < topologyKeys.length; i++) {
+        String key = topologyKeys[i] + CONNECTOR + entry.getValue().get(topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(path, topologyKeys[i]);

Review comment:
       I guess you just need the full path for the leaf node.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {

Review comment:
       Better to add one more check that, if the topologyDefInConfig does not start with DELIMITER, then it is invalid.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-679343209


   > Seems to be similar to the current Topology.java design. Why we need a new class?
   > If anything missed in Topology.java, we can add new methods there to read the tree and return the desired information.
   
   This class is purely to collect the instance configs and return to the users in the format they want. Helix has no logic involved in this part and we do not do any calculation. We considered topology, and the conclusion is that it will make both more complicated for now. Topology is mainly used for rebalancing purpose, and Helix adds default value if there is anything missing, and also Topology actually only supports two levels, as shown here:   
   
   public enum Types {
       ROOT,
       ZONE,
       INSTANCE
     }
   
   Although other levels keys are inserted in the tree, they're ignored during further calculation. This guideline is hard coded everywhere through that class. If we're going to change that, it's basically rewriting it.
   Another big issue is that Topology is tightly coupled with `faultZoneType` in different logic. But the new API totally ignore that field. 
   Although in the long run we may consolidate two classes, the prerequisite will be topology can support real multiple level (more than 2) hierarchy without hardcoding anything.
   Trie data structure is more suitable for this use case regarding simpleness and efficiency. This class will only be used when users query domain information for the cluster or for any specific domain.


----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r490534836



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       There're some offline discussion, and the conclusion is that trie is a good fit for cluster topology, as also mentioned here: https://github.com/apache/helix/issues/447
   and should apply to both rebalance and user facing APIs, meaning that we only need one class for both uses. 
   In this PR, we will create a generic trie structure with validation and construction logic. Topology APIs will use this structure and do whatever traverse needed. Laster, `topology` logic will also use this trie structure and implement the functions it need. The new trie structure is not fundamentally different from the tree in `topology` now, but with a clearer organization/validation logic and present in a more generic way. `topology` should be able to use it, if something mismatch, we can always come back to adjust the structure. 




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r488396626



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,247 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+  private static final String DELIMITER = "/";

Review comment:
       Can we just refer to the DELIMITER and CONNECTOR in ClusterTrie?

##########
File path: helix-core/src/main/java/org/apache/helix/HelixAdmin.java
##########
@@ -420,6 +420,24 @@ void addStateModelDef(String clusterName, String stateModelDef, StateModelDefini
    */
   void removeCloudConfig(String clusterName);
 
+  /**
+   * Get the topology of a specific cluster
+   * @param clusterName
+   */
+  Map<String, List<String>> getAllTopology(String clusterName);
+
+  /**
+   * Get all the instances under the fault zone
+   * @param clusterName
+   */
+  Map<String, List<String>> getInstancesUnderFaultZone(String clusterName);
+
+  /**
+   * Get all the instances whose domain config is not valid
+   * @param clusterName
+   */
+  List<String> getInvalidInstances(String clusterName);
+

Review comment:
       Didn't we agree to add one method that returns ClusterTopology object here?




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493049639



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);

Review comment:
       Commented in your new code. Let me resolve this one.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-685003252


   > Are we going to add HelixAdmin API changes to this PR?
   
   I plan to add in next PR. This one will have complete logic for trie topology. Admin API should be straightforward.


----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493066795



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       Yeah, I agree. Changed the name.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       Do you suggest to replace the _path in TrieNode with parent node field? I feel the _path is easier for traversal. Is the concern mainly on the memory usage?

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,155 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef = Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream()

Review comment:
       I see what you mean. I trimmed spaces after split. I think space should be allowed. Updated PR.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r491130388



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {

Review comment:
       I tried with a null value for domainType, but feel the logic is still very difficult to merge. 
   If you look at the two functions:
   ```
   private TrieNode getStartNode(LinkedHashMap<String, String> domainMap) {
       TrieNode curNode = _trieClusterTopology.getRootNode();
       TrieNode nextNode;
       for (Map.Entry<String, String> entry : domainMap.entrySet()) {
         nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
         if (nextNode == null) {
           throw new IllegalArgumentException(String
               .format("The input domain %s does not have the value %s", entry.getKey(),
                   entry.getValue()));
         }
         curNode = nextNode;
       }
       return curNode;
     }
   ```
   
   ```
     private List<TrieNode> getStartNodes(String domainType) {
       List<TrieNode> results = new ArrayList<>();
       TrieNode curNode = _trieClusterTopology.getRootNode();
       Deque<TrieNode> nodeStack = new ArrayDeque<>();
       nodeStack.push(curNode);
       while (!nodeStack.isEmpty()) {
         curNode = nodeStack.pop();
         if (curNode.getNodeKey().equals(domainType)) {
           results.add(curNode);
         } else {
           for (TrieNode child : curNode.getChildren().values()) {
             nodeStack.push(child);
           }
         }
       }
       return results;
     }
   ```
   
   They're not easy to combine, since one is to find a specific node in the trie, while the other is to find all nodes for a certain level across the trie. Thoughts?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-698102117


   This PR is ready to merge. Approved by @jiajunwang 
   Final commit message:
   Add TrieClusterTopology for retrieving hierarchical topology in a cluster.
   - Add TrieNode class to define a node in the trie
   - Add ClusterTrie class to handle the construction, validation and retrieval of nodes/paths in the trie.
   - Add ClusterTopology class to provide different APIs for users to retrieve cluster topology information.
   - Add APIs in HelixAdmin to retrieve ClusterTopology of a specific cluster.
   


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r486066806



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    if (domainType.equals(_topologyKeys[0])) {
+      return getClusterTopology();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < _topologyKeys.length; i++) {
+      if (_topologyKeys[i].equals(domainType)) {
+        parentDomainType = _topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return all the end nodes under fault zone type. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getInstancesUnderFaultZone() {
+    return getTopologyUnderDomainType(_faultZoneType);
+  }
+
+  private Map<String, String> convertPathToDomain(String path) {
+    Map<String, String> results = new HashMap<>();
+    for (String part : path.substring(1).split(DELIMITER)) {
+      results.put(part.substring(0, part.indexOf(CONNECTOR)),
+          part.substring(part.indexOf(CONNECTOR) + 1));
+    }
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _rootNode;
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {

Review comment:
       nit, this check is not necessary. The else condition will ensure an instance with an empty list to be removed.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = new TrieNode(new HashMap<>(), "", "ROOT");

Review comment:
       Can we let constructTrie() return a root which is constructed by the private method internally?
   This helps to simplify the code a little bit.

##########
File path: helix-core/src/main/java/org/apache/helix/HelixAdmin.java
##########
@@ -420,6 +420,41 @@ void addStateModelDef(String clusterName, String stateModelDef, StateModelDefini
    */
   void removeCloudConfig(String clusterName);
 
+  /**
+   * Get the topology of a specific cluster
+   * @param clusterName
+   */
+  Map<String, List<String>> getClusterTopology(String clusterName);

Review comment:
       I think adding 5 new methods to HelixAdmin is overkill. Can we just add a method that returns the ClusterTopology and then relies on the ClusterTopology class methods to return information?

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    if (domainType.equals(_topologyKeys[0])) {
+      return getClusterTopology();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < _topologyKeys.length; i++) {
+      if (_topologyKeys[i].equals(domainType)) {
+        parentDomainType = _topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return all the end nodes under fault zone type. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getInstancesUnderFaultZone() {
+    return getTopologyUnderDomainType(_faultZoneType);
+  }
+
+  private Map<String, String> convertPathToDomain(String path) {
+    Map<String, String> results = new HashMap<>();
+    for (String part : path.substring(1).split(DELIMITER)) {
+      results.put(part.substring(0, part.indexOf(CONNECTOR)),
+          part.substring(part.indexOf(CONNECTOR) + 1));
+    }
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _rootNode;
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);

Review comment:
       How are the callers supposed to know which instance has been removed because the configuration is invalid?
   In addition, the removal is directly modifying the input map. So it could dangerous since it may change the caller's map implicitly.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);

Review comment:
       I think we are not requiring such strict formatting elsewhere. For example, spaces are ignored.
   So maybe you should also trim here so as to ensure the logic is consistent.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();

Review comment:
       What if topology awareness is disabled on this cluster? Shall we still throw Exception?




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493979714



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       So it either String operation on the read side or on the write side. Maybe only write side would be better since we definitely have more reads.
   Let's keep the current design for now. It is an internal method so we can optimize if necessary.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r494026067



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,249 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, final ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(Collections.emptyMap());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domainMap A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domainMap) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domainMap);
+    TrieNode startNode = getNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domainMap) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domainMap == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domainMap.size(); i++) {
+      if (!domainMap.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domainMap.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getNode(LinkedHashMap<String, String> domainMap) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domainMap.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domainType) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getNodeKey().equals(domainType)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       Agree. Moved to cluster trie.

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       Yep, I've moved them to clustertrie.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r489578331



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);

Review comment:
       I trim the topologyDef, do you mean I need to trim something else?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-698154255


   > > This PR is ready to merge. Approved by @jiajunwang
   > > Final commit message:
   > > Add TrieClusterTopology for retrieving hierarchical topology in a cluster.
   > > 
   > > * Add TrieNode class to define a node in the trie
   > > * Add ClusterTrie class to handle the construction, validation and retrieval of nodes/paths in the trie.
   > > * Add ClusterTopology class to provide different APIs for users to retrieve cluster topology information.
   > > * Add APIs in HelixAdmin to retrieve ClusterTopology of a specific cluster.
   > 
   > Test result, please.
   
   @jiajunwang tested and updated.


----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-679405672


   Minor correction here. 
   I think the enum Type mentioned above is used only when cluster level clusterConfig.Topology is not defined. Otherwise, we still parse the topology definition in cluster config.
   
   In function getClusterTopologySetting:
   ```
   private static ClusterTopologyConfig getClusterTopologySetting(ClusterConfig clusterConfig) {
       ClusterTopologyConfig clusterTopologyConfig = new ClusterTopologyConfig();
          ...
         if (topologyDef != null) {
           // We parse "/zone/rack/sub-rack/instance" into  ["zone", "rack", "sub-rack", "instance"].
         } else {
           // Use default cluster topology definition, i,e. /root/zone/instance
           clusterTopologyConfig.endNodeType = Types.INSTANCE.name();
           clusterTopologyConfig.faultZoneType = Types.ZONE.name();
         }
         ...
     }
   ```
   
   
   In my understanding, current Topology.java has similar functions like validateAndOrderDomain and the constructor  TrieClusterTopology. 
   Topology::getClusterTopologySetting returns a parsed list of cluster topology as the constructor here. 
   There are similar logic of validating the instance topology config aligns with cluster's config. 
   
   IMHO, I personally prefer reusing current code. 


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493981376



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,249 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, final ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(Collections.emptyMap());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domainMap A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domainMap) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domainMap);
+    TrieNode startNode = getNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domainMap) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domainMap == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domainMap.size(); i++) {
+      if (!domainMap.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domainMap.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getNode(LinkedHashMap<String, String> domainMap) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domainMap.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domainType) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getNodeKey().equals(domainType)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       Shall we move them to ClusterTrie? They are useful to read the tree for all reader classes, right?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r485190915



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path);
+        }
+        curNode.addChild(key, nextNode);
+        curNode = nextNode;
+      }
+    }
+  }
+
+  private static class TrieNode {
+    // A mapping between trie key and children nodes.
+    private Map<String, TrieNode> _children;
+
+    // the complete path/prefix leading to the current node.
+    private final String _path;

Review comment:
       This API is for the common case where instances are returned for the fault zone type:
     public Map<String, List<String>> getInstancesUnderFaultZone() {
       return getTopologyUnderDomainType(_faultZoneType);
     }
   The name of the instance is implicitly embedded in the `path` field of the Trie node. e.g. the path could be like zone:myzone0/instance:myinstance0, so 'myinstance0' is the instance name.
   I didn't see the necessity of marking leaf nodes. Do you think it'll save time in querying? Because we always need to traverse to the leaves anyway.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493066795



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       Yeah, I agree. Changed the name.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       Do you suggest to replace the _path in TrieNode with parent node field? I feel the _path is easier for traversal. Is the concern mainly on the memory usage?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r491122851



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {

Review comment:
       I can merge these two, but I think it'll be a bit messy to combine the two cases, one is to find all nodes under a certain domain map, basically a branch of the trie, and the other is to find all nodes under a certain domain type, basically all branches of the trie.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r488313417



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    if (domainType.equals(_topologyKeys[0])) {
+      return getClusterTopology();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < _topologyKeys.length; i++) {
+      if (_topologyKeys[i].equals(domainType)) {
+        parentDomainType = _topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return all the end nodes under fault zone type. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getInstancesUnderFaultZone() {
+    return getTopologyUnderDomainType(_faultZoneType);
+  }
+
+  private Map<String, String> convertPathToDomain(String path) {
+    Map<String, String> results = new HashMap<>();
+    for (String part : path.substring(1).split(DELIMITER)) {
+      results.put(part.substring(0, part.indexOf(CONNECTOR)),
+          part.substring(part.indexOf(CONNECTOR) + 1));
+    }
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _rootNode;
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);

Review comment:
       Please see the updated PR. I add an invalid config list that customers can retrieve.




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-680243825


   > > Seems to be similar to the current Topology.java design. Why we need a new class?
   > > If anything missed in Topology.java, we can add new methods there to read the tree and return the desired information.
   > 
   > This class is purely to collect the instance configs and return to the users in the format they want. Helix has no logic involved in this part and we do not do any calculation. We considered topology, and the conclusion is that it will make both more complicated for now. Topology is mainly used for rebalancing purpose, and Helix adds default value if there is anything missing, and also Topology actually only supports two levels, as shown here:
   > 
   > public enum Types {
   > ROOT,
   > ZONE,
   > INSTANCE
   > }
   > 
   > Although other levels keys are inserted in the tree, they're ignored during further calculation. This guideline is hard coded everywhere through that class. If we're going to change that, it's basically rewriting it.
   > Another big issue is that Topology is tightly coupled with `faultZoneType` in different logic. But the new API totally ignore that field.
   > Although in the long run we may consolidate two classes, the prerequisite will be topology can support real multiple level (more than 2) hierarchy without hardcoding anything.
   > Trie data structure is more suitable for this use case regarding simpleness and efficiency. This class will only be used when users query domain information for the cluster or for any specific domain.
   
   As we discussed, let's define the detailed requirement first and then refine the java class design.
   But to answer some of you comment here, my premature reply would be,
   1. I think adding default value logic needs to be delivered to the user as well. They need to understand how Helix locates an instance. It is a Helix API eventually. If the rest API tells the user that this node has no definition, but internally, it is rebalanced as in the default fault zone, then it would be very confusing.
   2. Topology.java supports all possible Domain definition, not only 2 levels. Although the other layers are not used by the rebalancer, you can get the full tree structure from Topology.java if simply adding a method that traverses the tree like you what you are doing in this new class. I'm not aware of the necessity of rewriting.
   3. faultZoneType is purely optional.
   4. I agree that the trie data structure is more preferred. But it seems not a must for this requirement. If it is a normal tree, the get methods can still be developed, right?


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493051117



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       I understand the logic, I'm just asking about the name. Since this method can also be used to get the leaf, I feel getNode() is simpler and more generic. But no strong preference here.




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493054740



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {

Review comment:
       Yeah, I see. Let's keep the current design for now.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r491123519



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       The reason is that we don't know whether there're sub-domains with the same name, e.g. there can be /group:0/zone:0, /group:1/zone:0, /group:2/zone:0, we'll need full path to distinguish between them




----------------------------------------------------------------
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


[GitHub] [helix] alirezazamani merged pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
alirezazamani merged pull request #1307:
URL: https://github.com/apache/helix/pull/1307


   


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-698111830


   > This PR is ready to merge. Approved by @jiajunwang
   > Final commit message:
   > Add TrieClusterTopology for retrieving hierarchical topology in a cluster.
   > 
   > * Add TrieNode class to define a node in the trie
   > * Add ClusterTrie class to handle the construction, validation and retrieval of nodes/paths in the trie.
   > * Add ClusterTopology class to provide different APIs for users to retrieve cluster topology information.
   > * Add APIs in HelixAdmin to retrieve ClusterTopology of a specific cluster.
   
   Test result, please.


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-679325615


   Seems to be similar to the current Topology.java design. Why we need a new class?
   If anything missed in Topology.java, we can add new methods there to read the tree and return the desired information.


----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476662851



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       Thanks for the detailed reply.
   
   I think the previous 3 points can be achieved by adding a flag to Topology::ClusterTopologyConfig and maybe using non static 'DEFAULT_DOMAIN_PREFIX'. 
   For the point 4, 
   Please correct me if I am wrong, line 288 in constructTrie will also add a "default" node similar to 'clusterTopologyKeyDefaultValue'. Maybe we could read the tree and bypass default node. 
   
   
   In my own opinion, I think the question would be, do we construct topology tree in class Topology for these instances? If yes, its better to read from the existing tree. If not, I think it's ok to have another orthogonal data structure representing the hierarchy. 




----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476662851



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       Thanks for the detailed reply.
   
   I think the previous 3 points can be achieved by adding a flag to Topology::ClusterTopologyConfig and maybe using non static 'DEFAULT_DOMAIN_PREFIX'. 
   For the point 4, 
    Maybe we could read the tree and bypass default node (or throw exception when we find a default node in the path)
   
   
   In my own opinion, I think the final question would be, do we construct topology tree in class Topology for these instances? If yes, its better to read from the existing tree. If not, I think it's ok to have another orthogonal data structure representing the hierarchy. 




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493980956



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       Any thoughts about this comment, please?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-685936146


   > Are we going to add HelixAdmin API changes to this PR?
   
   @lei-xia I tried to add Helix admin related code in one PR. It would double the PR size, considering all the test cases. I feel it'll make the PR too huge for review purpose. Still think we should separate them. One is for the underlying logic, and the other for user facing API. What do you think?


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r490644166



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {

Review comment:
       nit, final for the ClusterConfig too?

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());

Review comment:
       nit, Collections.EMPTY_MAP?

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieNode.java
##########
@@ -0,0 +1,55 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.Map;
+
+
+public class TrieNode {
+  // A mapping between trie key and children nodes.
+  private Map<String, TrieNode> _children;
+
+  // the complete path/prefix leading to the current node.
+  private final String _path;
+
+  private final String _domainType;

Review comment:
       For generic, change to _key (node key)?

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       Seems the paths contain all the information starts from the root. Will it be too much duplicate information?
   Considering this object might be created frequently depends on the caller code, it could be better to just save the info about the current layer.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);

Review comment:
       So to summary the following comments, my suggestion would be,
   
   validateInstanceConfig(liveNodes, instanceConfigMap);
   _topologyKeys = getTopologyKeys(clusterConfig);
   _faultZoneType = clusterConfig.getFaultZoneType();
   _invalidInstances = getInvalidInstances(instanceConfigMap);
   instanceConfigMap.keyset().removeAll(_invalidInstances);
   _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
   
   Would this be cleaner and easier to maintain for the long term?

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {
+    List<TrieNode> results = new ArrayList<>();
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(curNode);
+    while (!nodeStack.isEmpty()) {
+      curNode = nodeStack.pop();
+      if (curNode.getDomainType().equals(domain)) {
+        results.add(curNode);
+      } else {
+        for (TrieNode child : curNode.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return results;
+  }

Review comment:
       I feel these methods can be put into the ClusterTrie class, given we can generalize the domain concept to the trie node keys. But we can also refactor later when we are migrating our existing Topology.java to use the same class.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieNode.java
##########
@@ -0,0 +1,55 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.Map;
+
+
+public class TrieNode {
+  // A mapping between trie key and children nodes.
+  private Map<String, TrieNode> _children;
+
+  // the complete path/prefix leading to the current node.
+  private final String _path;
+
+  private final String _domainType;
+
+  TrieNode(Map<String, TrieNode> children, String path, String domainType) {

Review comment:
       nit, I'm not sure if it is feasible given all the usages, but we can always create new children map inside the constructor instead of allowing input.

##########
File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
##########
@@ -1156,6 +1157,30 @@ public void removeCloudConfig(String clusterName) {
     accessor.removeProperty(keyBuilder.cloudConfig());
   }
 
+  @Override
+  public ClusterTopology getClusterTopology(String clusterName) {
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    String path = PropertyPathBuilder.instanceConfig(clusterName);
+    BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
+    List<ZNRecord> znRecords = baseAccessor.getChildren(path, null, 0, 0, 0);
+    for (ZNRecord record : znRecords) {
+      if (record != null) {
+        InstanceConfig instanceConfig = new InstanceConfig(record);
+        instanceConfigMap.put(instanceConfig.getInstanceName(), instanceConfig);
+      }
+    }
+    HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    List<LiveInstance> liveInstances = accessor.getChildValues(keyBuilder.liveInstances(), true);
+    List<String> liveNodes = new ArrayList<>();
+    liveInstances.forEach(liveInstance -> liveNodes.add(liveInstance.getInstanceName()));
+
+    ConfigAccessor configAccessor = new ConfigAccessor(_zkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);

Review comment:
       nit, but it would be cleaner if we use ZkBaseDataAccessor for all information access, IMHO.

##########
File path: helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
##########
@@ -1156,6 +1157,30 @@ public void removeCloudConfig(String clusterName) {
     accessor.removeProperty(keyBuilder.cloudConfig());
   }
 
+  @Override
+  public ClusterTopology getClusterTopology(String clusterName) {
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    String path = PropertyPathBuilder.instanceConfig(clusterName);
+    BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
+    List<ZNRecord> znRecords = baseAccessor.getChildren(path, null, 0, 0, 0);
+    for (ZNRecord record : znRecords) {
+      if (record != null) {
+        InstanceConfig instanceConfig = new InstanceConfig(record);
+        instanceConfigMap.put(instanceConfig.getInstanceName(), instanceConfig);
+      }
+    }
+    HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));

Review comment:
       Why we need 2 ZkBaseDataAccessors here?

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {

Review comment:
       Regardless of what I mentioned above, I think you can merge getPathUnderNode() and getStartNodes() to avoid duplicate logic. Some minor change is required, of course. But I think its a better option.
   
   High-level, you need a method that searches for multiple nodes under a certain scope. And another method that gets a certain node (getStartNode).

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {

Review comment:
       We may not need these 2 functions for supporting the public methods.
   I think what we want is a query method that accepts a Map of domain type and value pairs. However, the trick is that the value could be a wildcard "*" or just put null. Then we just need to maintain one set of logic. And for sure, no duplicate logic.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {

Review comment:
       Better to put _topologyKeys to the parameter. This is a better style since the private methods will not depend on each other.

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       nit, domain -> domainMap

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {

Review comment:
       In general, private methods are better to be "independent" with the object. So that maintenance of the class would be much easier.
   
   Suggest to change to
   private List<String> getInvalidIstanceConfig(Map<String, InstanceConfig> instanceConfigMap);
   
   Then we can do instanceConfigMap.keyset().removeAll(_invalidInstances) in the caller where the object fields are updated in a centralized place.

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _trieClusterTopology.getRootNode();
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private List<TrieNode> getStartNodes(String domain) {

Review comment:
       nit, domain -> domainType

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {

Review comment:
       Same here, better to put the _topologyKeys into the parameter list.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));

Review comment:
       instanceConfigMap.keyset().removeAll(_invalidInstances)

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {

Review comment:
       nit, just return the _topologyKeys as the return value. So we don't modify the object field secretly inside a "validate" method. Moreover, maybe call it getTopologyDef() would be more straightforward.

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       And, just curious, where does the "start" come from? The leaf node is a special case. But here the logic is generic, so it is just getNode(), right?




----------------------------------------------------------------
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


[GitHub] [helix] dasahcc commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476638953



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path);
+        }
+        curNode.addChild(key, nextNode);
+        curNode = nextNode;
+      }
+    }
+  }
+
+  private static class TrieNode {
+    // A mapping between trie key and children nodes.
+    private Map<String, TrieNode> _children;
+
+    // the complete path/prefix leading to the current node.
+    private final String _path;

Review comment:
       Shall we make some marks of leaf nodes? Then leaf nodes contains a list of instance names or something. Otherwise, it may not be very efficient for most of cases.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);

Review comment:
       If cluster config does not have topology, we should throw exception or give an empty return.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {

Review comment:
       Is this only for testing? The input is an empty map..




----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu edited a comment on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu edited a comment on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-679405672


   Minor correction here. 
   I think the enum Type mentioned above is used only when cluster level clusterConfig.Topology is not defined. Otherwise, we still parse the topology definition in cluster config.
   
   In function getClusterTopologySetting:
   ```
   private static ClusterTopologyConfig getClusterTopologySetting(ClusterConfig clusterConfig) {
       ClusterTopologyConfig clusterTopologyConfig = new ClusterTopologyConfig();
          ...
         if (topologyDef != null) {
           // We parse "/zone/rack/sub-rack/instance" into  ["zone", "rack", "sub-rack", "instance"].
         } else {
           // Use default cluster topology definition, i,e. /root/zone/instance
           clusterTopologyConfig.endNodeType = Types.INSTANCE.name();
           clusterTopologyConfig.faultZoneType = Types.ZONE.name();
         }
         ...
     }
   ```
   
   
   In my understanding, current Topology.java has similar functions like validateAndOrderDomain and the constructor  TrieClusterTopology. 
   Topology::getClusterTopologySetting returns a parsed list of cluster topology as the constructor here. 
   There are similar logic of validating the instance topology config aligns with cluster's config in computeInstanceTopologyMapHelper. 
   
   IMHO, I personally prefer reusing current code. Also I think its better to read from the tree since we already have the tree created for LB.
   
   Thanks,
   Xiaoyuan 


----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493048679



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,155 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef = Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream()

Review comment:
       The same example that I gave Xiaoyuan for the Topology.java change,
   
   "/group/ zone/ rack /host", do we accept this one? With your current logic, I think it will return "["group", " zone", " rack ", "host"]". Please mind the space in the keys. It will cause a mismatch in the later algorithm.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r491130388



##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {

Review comment:
       I tried with a null value for domainType, but feel the logic is still very difficult to merge. 
   If you look at the two functions:
   ///
   private TrieNode getStartNode(LinkedHashMap<String, String> domainMap) {
       TrieNode curNode = _trieClusterTopology.getRootNode();
       TrieNode nextNode;
       for (Map.Entry<String, String> entry : domainMap.entrySet()) {
         nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
         if (nextNode == null) {
           throw new IllegalArgumentException(String
               .format("The input domain %s does not have the value %s", entry.getKey(),
                   entry.getValue()));
         }
         curNode = nextNode;
       }
       return curNode;
     }
   
     private List<TrieNode> getStartNodes(String domainType) {
       List<TrieNode> results = new ArrayList<>();
       TrieNode curNode = _trieClusterTopology.getRootNode();
       Deque<TrieNode> nodeStack = new ArrayDeque<>();
       nodeStack.push(curNode);
       while (!nodeStack.isEmpty()) {
         curNode = nodeStack.pop();
         if (curNode.getNodeKey().equals(domainType)) {
           results.add(curNode);
         } else {
           for (TrieNode child : curNode.getChildren().values()) {
             nodeStack.push(child);
           }
         }
       }
       return results;
     }
   ///
   They're not easy to combine, since one is to find a specific node in the trie, while the other is to find all nodes for a certain level across the trie. Thoughts?




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r491123519



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       The reason is that we don't know whether there're sub-domains with the same name, e.g. there can be /group:0/zone:0, /group:1/zone:0, /group:2/zone:0, we'll need full path to distinguish between them.




----------------------------------------------------------------
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


[GitHub] [helix] jiajunwang commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r493048679



##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,155 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef = Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream()

Review comment:
       The same example that I gave Xiaoyuan for the Topology.java change,
   
   "/group/ zone/ rack /host", do we accept this one? With your current logic, I think it will return "["group", " zone", " rack ", "host"]". Please mind the space in the keys. It will cause a mismatch in the later algorithm.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,340 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = ":";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+  private final String _faultZoneType;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);

Review comment:
       Commented in your new code. Let me resolve this one.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       I don't get the reason. With your example, it should be very simple to distinguish since these 3 zone:0 nodes are under different parent nodes. And while you are traversing, the algorithm can construct the full path based on parents' information. You can even keep a stack with paths for this purpose.

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, List<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      results.put(startNode.getPath() + DELIMITER + child.getKey(),
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath()));
+    });
+    return results;
+  }
+
+  /**
+   * Return the full topology of a certain domain type.
+   * @param domainType a specific type of domain, e.g. zone
+   * @return the topology of the given domain type, e.g. {["/group:0/zone:0": {"rack:0/host:0",
+   * "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomainType(String domainType) {
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    if (domainType.equals(topologyKeys[0])) {
+      return getTopologyMap();
+    }
+    Map<String, List<String>> results = new HashMap<>();
+    String parentDomainType = null;
+    for (int i = 1; i < topologyKeys.length; i++) {
+      if (topologyKeys[i].equals(domainType)) {
+        parentDomainType = topologyKeys[i - 1];
+        break;
+      }
+    }
+    // get all the starting nodes for the domain type
+    List<TrieNode> startNodes = getStartNodes(parentDomainType);
+    for (TrieNode startNode : startNodes) {
+      results.putAll(getTopologyUnderPath(startNode.getPath()));
+    }
+    return results;
+  }
+
+  /**
+   * Return the topology under a certain path as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param path a path to a certain Trie node, e.g. /group:1/zone:2
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderPath(String path) {
+    Map<String, String> domain = convertPathToDomain(path);
+    return getTopologyUnderDomain(domain);
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    String[] topologyKeys = _trieClusterTopology.getTopologyKeys();
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", topologyKeys[i]));
+      } else {
+        orderedDomain.put(topologyKeys[i], domain.get(topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private List<String> truncatePath(Set<String> paths, String toRemovePath) {
+    List<String> results = new ArrayList<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {

Review comment:
       I understand the logic, I'm just asking about the name. Since this method can also be used to get the leaf, I feel getNode() is simpler and more generic. But no strong preference here.

##########
File path: helix-core/src/main/java/org/apache/helix/api/topology/ClusterTopology.java
##########
@@ -0,0 +1,248 @@
+package org.apache.helix.api.topology;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ClusterTrie;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.TrieNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.helix.model.ClusterTrie.CONNECTOR;
+import static org.apache.helix.model.ClusterTrie.DELIMITER;
+
+
+public class ClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(ClusterTopology.class);
+
+  private final ClusterTrie _trieClusterTopology;
+
+  public ClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    _trieClusterTopology = new ClusterTrie(liveNodes, instanceConfigMap, clusterConfig);
+  }
+
+  /**
+   * Return the whole topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a list of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/host, the result may be {
+   * ["/group:0": {"/zone:0/rack:0/host:0", "/zone:1/rack:1/host:1"}], ["/group:1": {"/zone:1
+   * /rack:1/host:1", "/zone:1/rack:1/host:2"}]}
+   */
+  public Map<String, List<String>> getTopologyMap() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return all the instances under fault zone type. The key of the returned map is each fault
+   * zone name, and the value is a list of string that represents the path to each end node in
+   * that fault zone.
+   * @return , e.g. if the fault zone is "zone", it may return {["/group:0/zone:0": {"rack:0/host
+   * :0", "rack:1/host:1"}, ["/group:0/zone:1": {"/rack:0:host:2", "/rack:1/host:3"}]}
+   */
+  public Map<String, List<String>> getFaultZoneMap() {
+    String faultZone = _trieClusterTopology.getFaultZoneType();
+    if (faultZone == null) {
+      throw new IllegalArgumentException("The fault zone in cluster config is not defined");
+    }
+    return getTopologyUnderDomainType(faultZone);
+  }
+
+  /**
+   * Return the instances whose domain field is not valid
+   */
+  public List<String> getInvalidInstances() {
+    return _trieClusterTopology.getInvalidInstances();
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a list of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["/group:1/zone:2/rack:0": {"/host:0",
+   * "/host:1"}, ["/group:1/zone:2/rack:1": {"/host:2", "/host:3"}]}
+   */
+  private Map<String, List<String>> getTopologyUnderDomain(Map<String, String> domain) {

Review comment:
       Yeah, I see. Let's keep the current design for now.

##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieNode.java
##########
@@ -0,0 +1,59 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class TrieNode {
+  // A mapping between trie key and children nodes.
+  private Map<String, TrieNode> _children;
+
+  // the complete path/prefix leading to the current node.
+  private final String _path;
+
+  private final String _nodeKey;
+
+  TrieNode(String path, String nodeKey) {
+    _path = path;
+    _nodeKey = nodeKey;
+    _children = new HashMap<>();
+  }
+
+  public Map<String, TrieNode> getChildren() {
+    return _children;
+  }
+
+  public String getPath() {
+    return _path;
+  }
+
+  public String getNodeKey() {
+    return _nodeKey;
+  }
+
+  public void addChildrenMap(Map <String, TrieNode> children) {

Review comment:
       I guess this is not necessary?

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())

Review comment:
       nit, the first tirm becomes unnecessary with the later one in the map().

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())
+            .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (topologyDef.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    return topologyDef;
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    TrieNode rootNode = new TrieNode("", "ROOT");
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";

Review comment:
       StringBuilder for the performance.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    topologyDef =
+        Arrays.asList(topologyDefInConfig.trim().split(DELIMITER)).stream().map(str -> str.trim())
+            .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (topologyDef.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+    return topologyDef;
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    TrieNode rootNode = new TrieNode("", "ROOT");
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < topologyKeys.length; i++) {
+        String key = topologyKeys[i] + CONNECTOR + entry.getValue().get(topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(path, topologyKeys[i]);

Review comment:
       I guess you just need the full path for the leaf node.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,156 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    _topologyKeys = getTopologyDef(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _invalidInstances = getInvalidInstancesFromConfig(instanceConfigMap, _topologyKeys);
+    instanceConfigMap.keySet().removeAll(_invalidInstances);
+    _rootNode = constructTrie(instanceConfigMap, _topologyKeys);
+  }
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  private List<String> getInvalidInstancesFromConfig(Map<String, InstanceConfig> instanceConfigMap,
+      final String[] topologyKeys) {
+    List<String> invalidInstances = new ArrayList<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        invalidInstances.add(instanceName);
+      }
+    }
+    return invalidInstances;
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private String[] getTopologyDef(ClusterConfig clusterConfig) {
+    String[] topologyDef;
+    String topologyDefInConfig = clusterConfig.getTopology();
+    if (topologyDefInConfig == null) {

Review comment:
       Better to add one more check that, if the topologyDefInConfig does not start with DELIMITER, then it is invalid.

##########
File path: helix-core/src/main/java/org/apache/helix/model/ClusterTrie.java
##########
@@ -0,0 +1,150 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class ClusterTrie {
+  public static final String DELIMITER = "/";
+  public static final String CONNECTOR = ":";
+
+  private static Logger logger = LoggerFactory.getLogger(ClusterTrie.class);
+  private TrieNode _rootNode;
+  private String[] _topologyKeys;
+  private String _faultZoneType;
+  private List<String> _invalidInstances = new ArrayList<>();
+
+  public ClusterTrie(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    validateInstanceConfig(liveNodes, instanceConfigMap);
+    validateClusterConfig(clusterConfig);
+    _faultZoneType = clusterConfig.getFaultZoneType();
+    _rootNode = constructTrie(instanceConfigMap);
+  }
+
+
+  public TrieNode getRootNode() {
+    return _rootNode;
+  }
+
+  public String[] getTopologyKeys() {
+    return _topologyKeys;
+  }
+
+  public  String getFaultZoneType() {
+    return _faultZoneType;
+  }
+
+  public List<String> getInvalidInstances() {
+    return _invalidInstances;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    for (String instanceName : instanceConfigMap.keySet()) {
+      try {
+        Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set", domainAsMap.get(key),
+                instanceName));
+            _invalidInstances.add(instanceName);
+            break;
+          }
+        }
+      } catch (IllegalArgumentException e) {
+        _invalidInstances.add(instanceName);
+      }
+    }
+    _invalidInstances.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  private void validateInstanceConfig(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      List<String> liveNodesCopy = new ArrayList<>();
+      liveNodesCopy.addAll(liveNodes);
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes
+              : liveNodesCopy.removeAll(instanceConfigMap.keySet())));
+    }
+  }
+
+  // Note that we do not validate whether topology-aware is enabled or fault zone type is
+  // defined, as they do not block the construction of the trie
+  private void validateClusterConfig(ClusterConfig clusterConfig) {
+    String topologyDef = clusterConfig.getTopology();
+    if (topologyDef == null) {
+      throw new HelixException(String.format("The topology of cluster %s is empty!",
+          clusterConfig.getClusterName()));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/host will return ["group", "zone", "rack", "host"].
+    _topologyKeys = Arrays.asList(topologyDef.trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    if (_topologyKeys.length == 0) {
+      throw new HelixException(String.format("The topology of cluster %s is not correctly defined",
+          clusterConfig.getClusterName()));
+    }
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private TrieNode constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    TrieNode rootNode = new TrieNode(new HashMap<>(), "", "ROOT");
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);
+        path = path + DELIMITER + key;
+        TrieNode nextNode = curNode.getChildren().get(key);
+        if (nextNode == null) {
+          nextNode = new TrieNode(new HashMap<>(), path, _topologyKeys[i]);

Review comment:
       It's both memory and additional computing for splitting/merging the path String.
   If we just put minimum info into trie, then there is no need to merge Strings. And there is no additional memory usage. Then when you read the tree, you don't need to split based on the ":" again. So I feel there would be a lot of saving on both the write and read side.
   
   To be specific, I think the node needs: 1. children map<domain value, TrieNode>, 2. domain Type (optional, since the recorded topology def list can be used to determine the type), 3. domain value (optional too, duplicate with the childrm map key, but could be convient).
   Then when you read, you can fill the type according to the depth of the tree and collect the domain value based on the children map key.
   The matter is to avoid String operations in which we create new String objects.




----------------------------------------------------------------
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


[GitHub] [helix] zhangmeng916 commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-698102117






----------------------------------------------------------------
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


[GitHub] [helix] lei-xia commented on pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
lei-xia commented on pull request #1307:
URL: https://github.com/apache/helix/pull/1307#issuecomment-684998974


   Are we going to add HelixAdmin API changes to this PR?


----------------------------------------------------------------
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


[GitHub] [helix] alirezazamani merged pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
alirezazamani merged pull request #1307:
URL: https://github.com/apache/helix/pull/1307


   


----------------------------------------------------------------
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


[GitHub] [helix] xyuanlu commented on a change in pull request #1307: Add TrieClusterTopology for retrieving hierarchical topology

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1307:
URL: https://github.com/apache/helix/pull/1307#discussion_r476662851



##########
File path: helix-core/src/main/java/org/apache/helix/model/TrieClusterTopology.java
##########
@@ -0,0 +1,244 @@
+package org.apache.helix.model;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is a class that uses a trie data structure to represent cluster topology. Each node
+ * except the terminal node represents a certain domain in the topology, and an terminal node
+ * represents an instance in the cluster.
+ */
+public class TrieClusterTopology {
+  private static Logger logger = LoggerFactory.getLogger(TrieClusterTopology.class);
+  private static final String DELIMITER = "/";
+  private static final String CONNECTOR = "_";
+
+  private final TrieNode _rootNode;
+  private final String[] _topologyKeys;
+
+  public TrieClusterTopology(final List<String> liveNodes,
+      final Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (instanceConfigMap == null || !instanceConfigMap.keySet().containsAll(liveNodes)) {
+      throw new HelixException(String.format("Config for instances %s is not found!",
+          instanceConfigMap == null ? liveNodes : liveNodes.removeAll(instanceConfigMap.keySet())));
+    }
+    // A list of all keys in cluster topology, e.g., a cluster topology defined as
+    // /group/zone/rack/instance will return ["group", "zone", "rack", "instance"].
+    _topologyKeys = Arrays.asList(clusterConfig.getTopology().trim().split(DELIMITER)).stream()
+        .filter(str -> !str.isEmpty()).collect(Collectors.toList()).toArray(new String[0]);
+    _rootNode = new TrieNode(new HashMap<>(), DELIMITER);
+    constructTrie(instanceConfigMap);
+  }
+
+  /**
+   * Return the topology of a cluster as a map. The key of the map is the first level of
+   * domain, and the value is a set of string that represents the path to each end node in that
+   * domain. E.g., assume the topology is defined as /group/zone/rack/instance, the result may be {
+   * ["group_0": {"zone_0/rack_0/instance_0", "zone_1/rack_1/instance_1"}], ["group_1": {"zone_1
+   * /rack_1/instance_1", "zone_1/rack_1/instance_2"}]}
+   */
+  public Map<String, Set<String>> getClusterTopology() {
+    return getTopologyUnderDomain(new HashMap<>());
+  }
+
+  /**
+   * Return the topology under a certain domain as a map. The key of the returned map is the next
+   * level domain, and the value is a set of string that represents the path to each end node in
+   * that domain.
+   * @param domain A map defining the domain name and its value, e.g. {["group": "1"], ["zone",
+   *               "2"]}
+   * @return the topology under the given domain, e.g. {["rack_0": {"instance_0", "instance_1"},
+   * ["rack_1": {"instance_2", "instance_3"}]}
+   */
+  public Map<String, Set<String>> getTopologyUnderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = validateAndOrderDomain(domain);
+    TrieNode startNode = getStartNode(orderedDomain);
+    Map<String, TrieNode> children = startNode.getChildren();
+    Map<String, Set<String>> results = new HashMap<>();
+    children.entrySet().forEach(child -> {
+      String key = child.getKey();
+      results.put(key,
+          truncatePath(getPathUnderNode(child.getValue()), child.getValue().getPath() + DELIMITER));
+    });
+    return results;
+  }
+
+  /**
+   * Validate the domain provided has continuous fields in cluster topology definition. If it
+   * has, order the domain based on cluster topology definition. E.g. if the cluster topology is
+   * /group/zone/rack/instance, and domain is provided as {["zone": "1"], ["group", "2"]} will be
+   * reordered in a LinkedinHashMap as {["group", "2"], ["zone": "1"]}
+   */
+  private LinkedHashMap<String, String> validateAndOrderDomain(Map<String, String> domain) {
+    LinkedHashMap<String, String> orderedDomain = new LinkedHashMap<>();
+    if (domain == null) {
+      throw new IllegalArgumentException("The domain should not be null");
+    }
+    for (int i = 0; i < domain.size(); i++) {
+      if (!domain.containsKey(_topologyKeys[i])) {
+        throw new IllegalArgumentException(String
+            .format("The input domain is not valid, the key %s is required", _topologyKeys[i]));
+      } else {
+        orderedDomain.put(_topologyKeys[i], domain.get(_topologyKeys[i]));
+      }
+    }
+    return orderedDomain;
+  }
+
+  /**
+   * Truncate each path in the given set and only retain path starting from current node's
+   * children to each end node.
+   * @param toRemovePath The path from root to current node. It should be removed so that users
+   *                     can get a better view.
+   */
+  private Set<String> truncatePath(Set<String> paths, String toRemovePath) {
+    Set<String> results = new HashSet<>();
+    paths.forEach(path -> {
+      String truncatedPath = path.replace(toRemovePath, "");
+      results.add(truncatedPath);
+    });
+    return results;
+  }
+
+  /**
+   * Return all the paths from a TrieNode as a set.
+   * @param node the node from where to collect all the nodes' paths.
+   * @return All the paths under the node.
+   */
+  private Set<String> getPathUnderNode(TrieNode node) {
+    Set<String> resultMap = new HashSet<>();
+    Deque<TrieNode> nodeStack = new ArrayDeque<>();
+    nodeStack.push(node);
+    while (!nodeStack.isEmpty()) {
+      node = nodeStack.pop();
+      if (node.getChildren().isEmpty()) {
+        resultMap.add(node.getPath());
+      } else {
+        for (TrieNode child : node.getChildren().values()) {
+          nodeStack.push(child);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  private TrieNode getStartNode(LinkedHashMap<String, String> domain) {
+    TrieNode curNode = _rootNode;
+    TrieNode nextNode;
+    for (Map.Entry<String, String> entry : domain.entrySet()) {
+      nextNode = curNode.getChildren().get(entry.getKey() + CONNECTOR + entry.getValue());
+      if (nextNode == null) {
+        throw new IllegalArgumentException(String
+            .format("The input domain %s does not have the value %s", entry.getKey(),
+                entry.getValue()));
+      }
+      curNode = nextNode;
+    }
+    return curNode;
+  }
+
+  private void removeInvalidInstanceConfig(Map<String, InstanceConfig> instanceConfigMap) {
+    Set<String> toRemoveConfig = new HashSet<>();
+    for (String instanceName : instanceConfigMap.keySet()) {
+      Map<String, String> domainAsMap = instanceConfigMap.get(instanceName).getDomainAsMap();
+      if (domainAsMap.isEmpty()) {
+        logger.info(String.format("Domain for instance %s is not set", instanceName));
+        toRemoveConfig.add(instanceName);
+      } else {
+        for (String key : _topologyKeys) {
+          String value = domainAsMap.get(key);
+          if (value == null || value.length() == 0) {
+            logger.info(String.format("Domain %s for instance %s is not set",
+                domainAsMap.get(key), instanceName));
+            toRemoveConfig.add(instanceName);
+            break;
+          }
+        }
+      }
+    }
+    toRemoveConfig.forEach(entry -> instanceConfigMap.remove(entry));
+  }
+
+  /**
+   * Constructs a trie based on the provided instance config map. It loops through all instance
+   * configs and constructs the trie in a top down manner.
+   */
+  private void constructTrie(Map<String, InstanceConfig> instanceConfigMap) {
+    removeInvalidInstanceConfig(instanceConfigMap);
+    Map<String, Map<String, String>> instanceDomainsMap = new HashMap<>();
+    instanceConfigMap.entrySet().forEach(
+        entry -> instanceDomainsMap.put(entry.getKey(), entry.getValue().getDomainAsMap()));
+
+    for (Map.Entry<String, Map<String, String>> entry : instanceDomainsMap.entrySet()) {
+      TrieNode curNode = _rootNode;
+      String path = "";
+      for (int i = 0; i < _topologyKeys.length; i++) {
+        String key = _topologyKeys[i] + CONNECTOR + entry.getValue().get(_topologyKeys[i]);

Review comment:
       Thanks for the detailed reply.
   
   I think the previous 3 points can be achieved by adding a flag to Topology::ClusterTopologyConfig and using non static 'DEFAULT_DOMAIN_PREFIX'. 
   For the point 4, 
   Please correct me if I am wrong, line 288 in constructTrie will also add a "default" node similar to 'clusterTopologyKeyDefaultValue'. Maybe we could read the tree and bypass default node. 
   
   
   In my own opinion, I think the question would be, do we construct topology tree in class Topology for these instances? If yes, its better to read from the existing tree. If not, I think it's ok to have another orthogonal data structure representing the hierarchy. 




----------------------------------------------------------------
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