You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by hu...@apache.org on 2020/04/01 22:47:03 UTC

[helix] 08/49: Add TrieRoutingData constructor (#731)

This is an automated email from the ASF dual-hosted git repository.

hulee pushed a commit to branch zooscalability
in repository https://gitbox.apache.org/repos/asf/helix.git

commit 9bf22694dfca070bc25d2b077b3f6a83c63ddf04
Author: Neal Sun <ne...@gmail.com>
AuthorDate: Mon Feb 10 16:41:11 2020 -0800

    Add TrieRoutingData constructor (#731)
    
    This PR adds a constructor to TrieRoutingData. The constructor takes in a mapping of sharding keys to realm addresses and parses it into a trie. Also, TrieRoutingData now treats sharding keys without leading slashes as invalid. Related changes are made due to the addition of the constructor: TrieNode is made private; missing logic is added for refreshRoutingData in ZkMetadataStoreDirectory.
---
 .../metadatastore/MetadataStoreRoutingData.java    |   7 +-
 .../helix/rest/metadatastore/TrieRoutingData.java  | 201 ++++++++++++---
 .../metadatastore/ZkMetadataStoreDirectory.java    |  36 ++-
 .../rest/metadatastore/TestTrieRoutingData.java    | 282 ++++++++++++++-------
 4 files changed, 380 insertions(+), 146 deletions(-)

diff --git a/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/MetadataStoreRoutingData.java b/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/MetadataStoreRoutingData.java
index 237e9b6..8d8b7e3 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/MetadataStoreRoutingData.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/MetadataStoreRoutingData.java
@@ -22,7 +22,6 @@ package org.apache.helix.rest.metadatastore;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
-
 public interface MetadataStoreRoutingData {
   /**
    * Given a path, return all the "metadata store sharding key-metadata store realm address" pairs
@@ -33,15 +32,17 @@ public interface MetadataStoreRoutingData {
    * @param path - the path where the search is conducted
    * @return all "sharding key-realm address" pairs where the sharding keys contain the given
    *         path if the path is valid; empty mapping otherwise
+   * @throws IllegalArgumentException - when the path is invalid
    */
-  Map<String, String> getAllMappingUnderPath(String path);
+  Map<String, String> getAllMappingUnderPath(String path) throws IllegalArgumentException;
 
   /**
    * Given a path, return the realm address corresponding to the sharding key contained in the
    * path. If the path doesn't contain a sharding key, throw NoSuchElementException.
    * @param path - the path where the search is conducted
    * @return the realm address corresponding to the sharding key contained in the path
+   * @throws IllegalArgumentException - when the path is invalid
    * @throws NoSuchElementException - when the path doesn't contain a sharding key
    */
-  String getMetadataStoreRealm(String path) throws NoSuchElementException;
+  String getMetadataStoreRealm(String path) throws IllegalArgumentException, NoSuchElementException;
 }
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/TrieRoutingData.java b/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/TrieRoutingData.java
index b89a5f9..28add4c 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/TrieRoutingData.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/TrieRoutingData.java
@@ -23,8 +23,10 @@ import java.util.ArrayDeque;
 import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import org.apache.helix.rest.metadatastore.exceptions.InvalidRoutingDataException;
 
 /**
  * This is a class that uses a data structure similar to trie to represent metadata store routing
@@ -37,15 +39,26 @@ public class TrieRoutingData implements MetadataStoreRoutingData {
 
   private final TrieNode _rootNode;
 
-  // TODO: THIS IS A TEMPORARY PLACEHOLDER. A proper constructor will be created, which will not
-  // take in a TrieNode; it instead initializes the rootNode and creates a trie based on
-  // some input data. The constructor is blocked by the implementation of RoutingDataAccessor, and
-  // will therefore be implemented later.
-  public TrieRoutingData(TrieNode rootNode) {
-    _rootNode = rootNode;
+  public TrieRoutingData(Map<String, List<String>> routingData) throws InvalidRoutingDataException {
+    if (routingData == null || routingData.isEmpty()) {
+      throw new InvalidRoutingDataException("routingData cannot be null or empty");
+    }
+
+    if (isRootShardingKey(routingData)) {
+      Map.Entry<String, List<String>> entry = routingData.entrySet().iterator().next();
+      _rootNode = new TrieNode(Collections.emptyMap(), "/", true, entry.getKey());
+    } else {
+      _rootNode = new TrieNode(new HashMap<>(), "/", false, "");
+      constructTrie(routingData);
+    }
   }
 
-  public Map<String, String> getAllMappingUnderPath(String path) {
+  public Map<String, String> getAllMappingUnderPath(String path) throws IllegalArgumentException {
+    if (path.isEmpty() || !path.substring(0, 1).equals(DELIMITER)) {
+      throw new IllegalArgumentException("Provided path is empty or does not have a leading \""
+          + DELIMITER + "\" character: " + path);
+    }
+
     TrieNode curNode;
     try {
       curNode = findTrieNode(path, false);
@@ -58,10 +71,10 @@ public class TrieRoutingData implements MetadataStoreRoutingData {
     nodeStack.push(curNode);
     while (!nodeStack.isEmpty()) {
       curNode = nodeStack.pop();
-      if (curNode._isLeaf) {
-        resultMap.put(curNode._name, curNode._realmAddress);
+      if (curNode.isShardingKey()) {
+        resultMap.put(curNode.getPath(), curNode.getRealmAddress());
       } else {
-        for (TrieNode child : curNode._children.values()) {
+        for (TrieNode child : curNode.getChildren().values()) {
           nodeStack.push(child);
         }
       }
@@ -69,9 +82,15 @@ public class TrieRoutingData implements MetadataStoreRoutingData {
     return resultMap;
   }
 
-  public String getMetadataStoreRealm(String path) throws NoSuchElementException {
+  public String getMetadataStoreRealm(String path)
+      throws IllegalArgumentException, NoSuchElementException {
+    if (path.isEmpty() || !path.substring(0, 1).equals(DELIMITER)) {
+      throw new IllegalArgumentException("Provided path is empty or does not have a leading \""
+          + DELIMITER + "\" character: " + path);
+    }
+
     TrieNode leafNode = findTrieNode(path, true);
-    return leafNode._realmAddress;
+    return leafNode.getRealmAddress();
   }
 
   /**
@@ -88,35 +107,28 @@ public class TrieRoutingData implements MetadataStoreRoutingData {
    */
   private TrieNode findTrieNode(String path, boolean findLeafAlongPath)
       throws NoSuchElementException {
-    if (path.equals(DELIMITER) || path.equals("")) {
-      if (findLeafAlongPath && !_rootNode._isLeaf) {
+    if (path.equals(DELIMITER)) {
+      if (findLeafAlongPath && !_rootNode.isShardingKey()) {
         throw new NoSuchElementException("No leaf node found along the path. Path: " + path);
       }
       return _rootNode;
     }
 
-    String[] splitPath;
-    if (path.substring(0, 1).equals(DELIMITER)) {
-      splitPath = path.substring(1).split(DELIMITER, 0);
-    } else {
-      splitPath = path.split(DELIMITER, 0);
-    }
-
     TrieNode curNode = _rootNode;
-    if (findLeafAlongPath && curNode._isLeaf) {
+    if (findLeafAlongPath && curNode.isShardingKey()) {
       return curNode;
     }
-    Map<String, TrieNode> curChildren = curNode._children;
-    for (String pathSection : splitPath) {
+    Map<String, TrieNode> curChildren = curNode.getChildren();
+    for (String pathSection : path.substring(1).split(DELIMITER, 0)) {
       curNode = curChildren.get(pathSection);
       if (curNode == null) {
         throw new NoSuchElementException(
             "The provided path is missing from the trie. Path: " + path);
       }
-      if (findLeafAlongPath && curNode._isLeaf) {
+      if (findLeafAlongPath && curNode.isShardingKey()) {
         return curNode;
       }
-      curChildren = curNode._children;
+      curChildren = curNode.getChildren();
     }
     if (findLeafAlongPath) {
       throw new NoSuchElementException("No leaf node found along the path. Path: " + path);
@@ -124,36 +136,145 @@ public class TrieRoutingData implements MetadataStoreRoutingData {
     return curNode;
   }
 
-  // TODO: THE CLASS WILL BE CHANGED TO PRIVATE ONCE THE CONSTRUCTOR IS CREATED.
-  static class TrieNode {
+  /**
+   * Checks for the edge case when the only sharding key in provided routing data is the delimiter
+   * or an empty string. When this is the case, the trie is valid and contains only one node, which
+   * is the root node, and the root node is a leaf node with a realm address associated with it.
+   * @param routingData - a mapping from "sharding keys" to "realm addresses" to be parsed into a
+   *          trie
+   * @return whether the edge case is true
+   */
+  private boolean isRootShardingKey(Map<String, List<String>> routingData) {
+    if (routingData.size() == 1) {
+      for (List<String> shardingKeys : routingData.values()) {
+        return shardingKeys.size() == 1 && shardingKeys.get(0).equals(DELIMITER);
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Constructs a trie based on the provided routing data. It loops through all sharding keys and
+   * constructs the trie in a top down manner.
+   * @param routingData- a mapping from "sharding keys" to "realm addresses" to be parsed into a
+   *          trie
+   * @throws InvalidRoutingDataException - when there is an empty sharding key (edge case that
+   *           always renders the routing data invalid); when there is a sharding key which already
+   *           contains a sharding key (invalid); when there is a sharding key that is a part of
+   *           another sharding key (invalid)
+   */
+  private void constructTrie(Map<String, List<String>> routingData)
+      throws InvalidRoutingDataException {
+    for (Map.Entry<String, List<String>> entry : routingData.entrySet()) {
+      if (entry.getValue().isEmpty()) {
+        throw new InvalidRoutingDataException(
+            "Realm address does not have associating sharding keys: " + entry.getKey());
+      }
+      for (String shardingKey : entry.getValue()) {
+        // Missing leading delimiter is invalid
+        if (shardingKey.isEmpty() || !shardingKey.substring(0, 1).equals(DELIMITER)) {
+          throw new InvalidRoutingDataException("Sharding key does not have a leading \""
+              + DELIMITER + "\" character: " + shardingKey);
+        }
+
+        // Root can only be a sharding key if it's the only sharding key. Since this method is
+        // running, the special case has already been checked, therefore it's definitely invalid
+        if (shardingKey.equals(DELIMITER)) {
+          throw new InvalidRoutingDataException(
+              "There exist other sharding keys. Root cannot be a sharding key.");
+        }
+
+        // Locate the next delimiter
+        int nextDelimiterIndex = shardingKey.indexOf(DELIMITER, 1);
+        int prevDelimiterIndex = 0;
+        String keySection = shardingKey.substring(prevDelimiterIndex + 1,
+            nextDelimiterIndex > 0 ? nextDelimiterIndex : shardingKey.length());
+        TrieNode curNode = _rootNode;
+        TrieNode nextNode = curNode.getChildren().get(keySection);
+
+        // If the key section is not the last section yet, go in the loop; if the key section is the
+        // last section, exit
+        while (nextDelimiterIndex > 0) {
+          // If the node is already a leaf node, the current sharding key is invalid; if the node
+          // doesn't exist, construct a node and continue
+          if (nextNode != null && nextNode.isShardingKey()) {
+            throw new InvalidRoutingDataException(shardingKey + " cannot be a sharding key because "
+                + shardingKey.substring(0, nextDelimiterIndex)
+                + " is its parent key and is also a sharding key.");
+          } else if (nextNode == null) {
+            nextNode = new TrieNode(new HashMap<>(), shardingKey.substring(0, nextDelimiterIndex),
+                false, "");
+            curNode.addChild(keySection, nextNode);
+          }
+          prevDelimiterIndex = nextDelimiterIndex;
+          nextDelimiterIndex = shardingKey.indexOf(DELIMITER, prevDelimiterIndex + 1);
+          keySection = shardingKey.substring(prevDelimiterIndex + 1,
+              nextDelimiterIndex > 0 ? nextDelimiterIndex : shardingKey.length());
+          curNode = nextNode;
+          nextNode = curNode.getChildren().get(keySection);
+        }
+
+        // If the last node already exists, it's a part of another sharding key, making the current
+        // sharding key invalid
+        if (nextNode != null) {
+          throw new InvalidRoutingDataException(shardingKey
+              + " cannot be a sharding key because it is a parent key to another sharding key.");
+        }
+        nextNode = new TrieNode(new HashMap<>(), shardingKey, true, entry.getKey());
+        curNode.addChild(keySection, nextNode);
+      }
+    }
+  }
+
+  private static class TrieNode {
     /**
      * This field is a mapping between trie key and children nodes. For example, node "a" has
      * children "ab" and "ac", therefore the keys are "b" and "c" respectively.
      */
-    Map<String, TrieNode> _children;
+    private Map<String, TrieNode> _children;
     /**
-     * This field means if the node is a terminal node in the tree sense, not the trie sense. Any
-     * node that has children cannot possibly be a leaf node because only the node without children
-     * can store information. If a node is leaf, then it shouldn't have any children.
+     * This field states whether the path represented by the node is a sharding key
      */
-    final boolean _isLeaf;
+    private final boolean _isShardingKey;
     /**
-     * This field aligns the traditional trie design: it entails the complete path/prefix leading to
-     * the current node. For example, the name of root node is "/", then the name of its child node
+     * This field contains the complete path/prefix leading to the current node. For example, the
+     * name of root node is "/", then the name of its child node
      * is "/a", and the name of the child's child node is "/a/b".
      */
-    final String _name;
+    private final String _path;
     /**
      * This field represents the data contained in a node(which represents a path), and is only
      * available to the terminal nodes.
      */
-    final String _realmAddress;
+    private final String _realmAddress;
 
-    TrieNode(Map<String, TrieNode> children, String name, boolean isLeaf, String realmAddress) {
+    TrieNode(Map<String, TrieNode> children, String path, boolean isShardingKey,
+        String realmAddress) {
       _children = children;
-      _isLeaf = isLeaf;
-      _name = name;
+      _isShardingKey = isShardingKey;
+      _path = path;
       _realmAddress = realmAddress;
     }
+
+    public Map<String, TrieNode> getChildren() {
+      return _children;
+    }
+
+    public boolean isShardingKey() {
+      return _isShardingKey;
+    }
+
+    public String getPath() {
+      return _path;
+    }
+
+    public String getRealmAddress() {
+      return _realmAddress;
+    }
+
+    public void addChild(String key, TrieNode node) {
+      _children.put(key, node);
+    }
   }
 }
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java b/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java
index 85f8f4a..5a88ca9 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/metadatastore/ZkMetadataStoreDirectory.java
@@ -42,9 +42,9 @@ import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 /**
- * ZK-based MetadataStoreDirectory that listens on the routing data in routing ZKs with a update callback.
+ * ZK-based MetadataStoreDirectory that listens on the routing data in routing ZKs with a update
+ * callback.
  */
 public class ZkMetadataStoreDirectory implements MetadataStoreDirectory, RoutingDataListener {
   private static final Logger LOG = LoggerFactory.getLogger(ZkMetadataStoreDirectory.class);
@@ -156,34 +156,42 @@ public class ZkMetadataStoreDirectory implements MetadataStoreDirectory, Routing
 
   /**
    * Callback for updating the cached routing data.
-   * Note: this method should not synchronize on the class or the map. We do not want namespaces blocking each other.
+   * Note: this method should not synchronize on the class or the map. We do not want namespaces
+   * blocking each other.
    * Threadsafe map is used for _realmToShardingKeysMap.
-   * The global consistency of the in-memory routing data is not a requirement (eventual consistency is enough).
+   * The global consistency of the in-memory routing data is not a requirement (eventual consistency
+   * is enough).
    * @param namespace
    */
   @Override
   public void refreshRoutingData(String namespace) {
-    // Safe to ignore the callback if routingDataMap is null.
+    // Safe to ignore the callback if any of the mapping is null.
     // If routingDataMap is null, then it will be populated by the constructor anyway
     // If routingDataMap is not null, then it's safe for the callback function to update it
+    if (_routingZkAddressMap == null || _routingDataMap == null
+        || _realmToShardingKeysMap == null) {
+      LOG.error("Construction is not completed! ");
+      return;
+    }
 
     // Check if namespace exists; otherwise, return as a NOP and log it
     if (!_routingZkAddressMap.containsKey(namespace)) {
-      LOG.error("Failed to refresh internally-cached routing data! Namespace not found: " + namespace);
+      LOG.error("Failed to refresh internally-cached routing data! Namespace not found: {}",
+          namespace);
+      return;
     }
 
     try {
-      _realmToShardingKeysMap.put(namespace, _routingDataReaderMap.get(namespace).getRoutingData());
+      Map<String, List<String>> rawRoutingData =
+          _routingDataReaderMap.get(namespace).getRoutingData();
+      _realmToShardingKeysMap.put(namespace, rawRoutingData);
+
+      MetadataStoreRoutingData routingData = new TrieRoutingData(rawRoutingData);
+      _routingDataMap.put(namespace, routingData);
     } catch (InvalidRoutingDataException e) {
-      LOG.error("Failed to get routing data for namespace: " + namespace + "!");
+      LOG.error("Failed to refresh cached routing data for namespace {}", namespace, e);
     }
 
-    if (_routingDataMap != null) {
-      MetadataStoreRoutingData newRoutingData =
-          new TrieRoutingData(new TrieRoutingData.TrieNode(null, null, false, null));
-      // TODO call constructRoutingData() here.
-      _routingDataMap.put(namespace, newRoutingData);
-    }
   }
 
   @Override
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/metadatastore/TestTrieRoutingData.java b/helix-rest/src/test/java/org/apache/helix/rest/metadatastore/TestTrieRoutingData.java
index 1b1754d..bf71456 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/metadatastore/TestTrieRoutingData.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/metadatastore/TestTrieRoutingData.java
@@ -19,146 +19,250 @@ package org.apache.helix.rest.metadatastore;
  * under the License.
  */
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import org.apache.helix.rest.metadatastore.exceptions.InvalidRoutingDataException;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestTrieRoutingData {
-  // TODO: add constructor related tests after constructor is finished
+  private TrieRoutingData _trie;
 
   @Test
-  public void testGetAllMappingUnderPathFromRoot() {
-    TrieRoutingData trie = constructTestTrie();
-    Map<String, String> result = trie.getAllMappingUnderPath("/");
-    Assert.assertEquals(result.size(), 4);
-    Assert.assertEquals(result.get("/b/c/d"), "realmAddressD");
-    Assert.assertEquals(result.get("/b/c/e"), "realmAddressE");
-    Assert.assertEquals(result.get("/b/f"), "realmAddressF");
-    Assert.assertEquals(result.get("/g"), "realmAddressG");
+  public void testConstructionMissingRoutingData() {
+    try {
+      new TrieRoutingData(null);
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(e.getMessage().contains("routingData cannot be null or empty"));
+    }
+    try {
+      new TrieRoutingData(Collections.emptyMap());
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(e.getMessage().contains("routingData cannot be null or empty"));
+    }
+  }
+
+  /**
+   * This test case is for the situation when there's only one sharding key and it's root.
+   */
+  @Test
+  public void testConstructionSpecialCase() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress", Collections.singletonList("/"));
+    TrieRoutingData trie;
+    try {
+      trie = new TrieRoutingData(routingData);
+      Map<String, String> result = trie.getAllMappingUnderPath("/");
+      Assert.assertEquals(result.size(), 1);
+      Assert.assertEquals(result.get("/"), "realmAddress");
+    } catch (InvalidRoutingDataException e) {
+      Assert.fail("Not expecting InvalidRoutingDataException");
+    }
+  }
+
+  @Test
+  public void testConstructionEmptyShardingKeys() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress1", Collections.emptyList());
+    try {
+      new TrieRoutingData(routingData);
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains("Realm address does not have associating sharding keys: realmAddress1"));
+    }
+  }
+
+  @Test
+  public void testConstructionShardingKeyNoLeadingSlash() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress1", Arrays.asList("/g", "/h/i", "/h/j"));
+    routingData.put("realmAddress2", Arrays.asList("b/c/d", "/b/f"));
+    routingData.put("realmAddress3", Collections.singletonList("/b/c/e"));
+    try {
+      new TrieRoutingData(routingData);
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(
+          e.getMessage().contains("Sharding key does not have a leading \"/\" character: b/c/d"));
+    }
+  }
+
+  @Test
+  public void testConstructionRootAsShardingKeyInvalid() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress1", Arrays.asList("/a/b", "/"));
+    try {
+      new TrieRoutingData(routingData);
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains("There exist other sharding keys. Root cannot be a sharding key."));
+    }
   }
 
   @Test
-  public void testGetAllMappingUnderPathFromRootEmptyPath() {
-    TrieRoutingData trie = constructTestTrie();
-    Map<String, String> result = trie.getAllMappingUnderPath("");
-    Assert.assertEquals(result.size(), 4);
-    Assert.assertEquals(result.get("/b/c/d"), "realmAddressD");
-    Assert.assertEquals(result.get("/b/c/e"), "realmAddressE");
-    Assert.assertEquals(result.get("/b/f"), "realmAddressF");
-    Assert.assertEquals(result.get("/g"), "realmAddressG");
+  public void testConstructionShardingKeyContainsAnother() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress1", Arrays.asList("/a/b", "/a/b/c"));
+    try {
+      new TrieRoutingData(routingData);
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "/a/b/c cannot be a sharding key because /a/b is its parent key and is also a sharding key."));
+    }
   }
 
   @Test
+  public void testConstructionShardingKeyIsAPartOfAnother() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress1", Arrays.asList("/a/b/c", "/a/b"));
+    try {
+      new TrieRoutingData(routingData);
+      Assert.fail("Expecting InvalidRoutingDataException");
+    } catch (InvalidRoutingDataException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "/a/b cannot be a sharding key because it is a parent key to another sharding key."));
+    }
+  }
+
+  /**
+   * Constructing a trie that will also be reused for other tests
+   * -----<empty>
+   * ------/-|--\
+   * -----b--g--h
+   * ----/-\---/-\
+   * ---c--f--i--j
+   * --/-\
+   * -d--e
+   * Note: "g", "i", "j" lead to "realmAddress1"; "d", "f" lead to "realmAddress2"; "e" leads to
+   * "realmAddress3"
+   */
+  @Test
+  public void testConstructionNormal() {
+    Map<String, List<String>> routingData = new HashMap<>();
+    routingData.put("realmAddress1", Arrays.asList("/g", "/h/i", "/h/j"));
+    routingData.put("realmAddress2", Arrays.asList("/b/c/d", "/b/f"));
+    routingData.put("realmAddress3", Collections.singletonList("/b/c/e"));
+    try {
+      _trie = new TrieRoutingData(routingData);
+    } catch (InvalidRoutingDataException e) {
+      Assert.fail("Not expecting InvalidRoutingDataException");
+    }
+  }
+
+  @Test(dependsOnMethods = "testConstructionNormal")
+  public void testGetAllMappingUnderPathEmptyPath() {
+    try {
+      _trie.getAllMappingUnderPath("");
+      Assert.fail("Expecting IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains("Provided path is empty or does not have a leading \"/\" character: "));
+    }
+  }
+
+  @Test(dependsOnMethods = "testConstructionNormal")
+  public void testGetAllMappingUnderPathNoLeadingSlash() {
+    try {
+      _trie.getAllMappingUnderPath("test");
+      Assert.fail("Expecting IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains("Provided path is empty or does not have a leading \"/\" character: test"));
+    }
+  }
+
+  @Test(dependsOnMethods = "testConstructionNormal")
+  public void testGetAllMappingUnderPathFromRoot() {
+    Map<String, String> result = _trie.getAllMappingUnderPath("/");
+    Assert.assertEquals(result.size(), 6);
+    Assert.assertEquals(result.get("/b/c/d"), "realmAddress2");
+    Assert.assertEquals(result.get("/b/c/e"), "realmAddress3");
+    Assert.assertEquals(result.get("/b/f"), "realmAddress2");
+    Assert.assertEquals(result.get("/g"), "realmAddress1");
+    Assert.assertEquals(result.get("/h/i"), "realmAddress1");
+    Assert.assertEquals(result.get("/h/j"), "realmAddress1");
+  }
+
+  @Test(dependsOnMethods = "testConstructionNormal")
   public void testGetAllMappingUnderPathFromSecondLevel() {
-    TrieRoutingData trie = constructTestTrie();
-    Map<String, String> result = trie.getAllMappingUnderPath("/b");
+    Map<String, String> result = _trie.getAllMappingUnderPath("/b");
     Assert.assertEquals(result.size(), 3);
-    Assert.assertEquals(result.get("/b/c/d"), "realmAddressD");
-    Assert.assertEquals(result.get("/b/c/e"), "realmAddressE");
-    Assert.assertEquals(result.get("/b/f"), "realmAddressF");
+    Assert.assertEquals(result.get("/b/c/d"), "realmAddress2");
+    Assert.assertEquals(result.get("/b/c/e"), "realmAddress3");
+    Assert.assertEquals(result.get("/b/f"), "realmAddress2");
   }
 
-  @Test
+  @Test(dependsOnMethods = "testConstructionNormal")
   public void testGetAllMappingUnderPathFromLeaf() {
-    TrieRoutingData trie = constructTestTrie();
-    Map<String, String> result = trie.getAllMappingUnderPath("/b/c/d");
+    Map<String, String> result = _trie.getAllMappingUnderPath("/b/c/d");
     Assert.assertEquals(result.size(), 1);
-    Assert.assertEquals(result.get("/b/c/d"), "realmAddressD");
+    Assert.assertEquals(result.get("/b/c/d"), "realmAddress2");
   }
 
-  @Test
+  @Test(dependsOnMethods = "testConstructionNormal")
   public void testGetAllMappingUnderPathWrongPath() {
-    TrieRoutingData trie = constructTestTrie();
-    Map<String, String> result = trie.getAllMappingUnderPath("/b/c/d/g");
+    Map<String, String> result = _trie.getAllMappingUnderPath("/b/c/d/g");
     Assert.assertEquals(result.size(), 0);
   }
 
-  @Test
-  public void testGetMetadataStoreRealm() {
-    TrieRoutingData trie = constructTestTrie();
+  @Test(dependsOnMethods = "testConstructionNormal")
+  public void testGetMetadataStoreRealmEmptyPath() {
     try {
-      Assert.assertEquals(trie.getMetadataStoreRealm("/b/c/d/x/y/z"), "realmAddressD");
-    } catch (NoSuchElementException e) {
-      Assert.fail("Not expecting NoSuchElementException");
+      Assert.assertEquals(_trie.getMetadataStoreRealm(""), "realmAddress2");
+      Assert.fail("Expecting IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains("Provided path is empty or does not have a leading \"/\" character: "));
     }
   }
 
-  @Test
+  @Test(dependsOnMethods = "testConstructionNormal")
   public void testGetMetadataStoreRealmNoSlash() {
-    TrieRoutingData trie = constructTestTrie();
     try {
-      Assert.assertEquals(trie.getMetadataStoreRealm("b/c/d/x/y/z"), "realmAddressD");
+      Assert.assertEquals(_trie.getMetadataStoreRealm("b/c/d/x/y/z"), "realmAddress2");
+      Assert.fail("Expecting IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Provided path is empty or does not have a leading \"/\" character: b/c/d/x/y/z"));
+    }
+  }
+
+  @Test(dependsOnMethods = "testConstructionNormal")
+  public void testGetMetadataStoreRealm() {
+    try {
+      Assert.assertEquals(_trie.getMetadataStoreRealm("/b/c/d/x/y/z"), "realmAddress2");
     } catch (NoSuchElementException e) {
       Assert.fail("Not expecting NoSuchElementException");
     }
   }
 
-  @Test
+  @Test(dependsOnMethods = "testConstructionNormal")
   public void testGetMetadataStoreRealmWrongPath() {
-    TrieRoutingData trie = constructTestTrie();
     try {
-      trie.getMetadataStoreRealm("/x/y/z");
+      _trie.getMetadataStoreRealm("/x/y/z");
       Assert.fail("Expecting NoSuchElementException");
     } catch (NoSuchElementException e) {
-      Assert.assertTrue(e.getMessage().contains("The provided path is missing from the trie. Path: /x/y/z"));
+      Assert.assertTrue(
+          e.getMessage().contains("The provided path is missing from the trie. Path: /x/y/z"));
     }
   }
 
-  @Test
+  @Test(dependsOnMethods = "testConstructionNormal")
   public void testGetMetadataStoreRealmNoLeaf() {
-    TrieRoutingData trie = constructTestTrie();
     try {
-      trie.getMetadataStoreRealm("/b/c");
+      _trie.getMetadataStoreRealm("/b/c");
       Assert.fail("Expecting NoSuchElementException");
     } catch (NoSuchElementException e) {
       Assert.assertTrue(e.getMessage().contains("No leaf node found along the path. Path: /b/c"));
     }
   }
-
-  /**
-   * Constructing a trie for testing purposes
-   * -----<empty>
-   * ------/--\
-   * -----b---g
-   * ----/-\
-   * ---c--f
-   * --/-\
-   * -d--e
-   */
-  private TrieRoutingData constructTestTrie() {
-    TrieRoutingData.TrieNode nodeD =
-        new TrieRoutingData.TrieNode(Collections.emptyMap(), "/b/c/d", true, "realmAddressD");
-    TrieRoutingData.TrieNode nodeE =
-        new TrieRoutingData.TrieNode(Collections.emptyMap(), "/b/c/e", true, "realmAddressE");
-    TrieRoutingData.TrieNode nodeF =
-        new TrieRoutingData.TrieNode(Collections.emptyMap(), "/b/f", true, "realmAddressF");
-    TrieRoutingData.TrieNode nodeG =
-        new TrieRoutingData.TrieNode(Collections.emptyMap(), "/g", true, "realmAddressG");
-    TrieRoutingData.TrieNode nodeC =
-        new TrieRoutingData.TrieNode(new HashMap<String, TrieRoutingData.TrieNode>() {
-          {
-            put("d", nodeD);
-            put("e", nodeE);
-          }
-        }, "c", false, "");
-    TrieRoutingData.TrieNode nodeB =
-        new TrieRoutingData.TrieNode(new HashMap<String, TrieRoutingData.TrieNode>() {
-          {
-            put("c", nodeC);
-            put("f", nodeF);
-          }
-        }, "b", false, "");
-    TrieRoutingData.TrieNode root =
-        new TrieRoutingData.TrieNode(new HashMap<String, TrieRoutingData.TrieNode>() {
-          {
-            put("b", nodeB);
-            put("g", nodeG);
-          }
-        }, "", false, "");
-
-    return new TrieRoutingData(root);
-  }
 }