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/08/14 18:13:56 UTC

[helix] 01/12: Implement RoutingDataManager to replace HttpRoutingDataReader

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

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

commit 5e85638aa8b2ef8c9f8f17da88b8a3cfdaa37534
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Mon Jul 20 23:46:15 2020 -0700

    Implement RoutingDataManager to replace HttpRoutingDataReader
    
    We want to make the routing data source configurable. As such, using HttpRoutingDataReader as the static Singleton that fetches and caches the routing data is no longer appropriate. This change adds an implementation of RoutingDataManager and the new RoutingDataReader interface, with HttpRoutingDataReader as one of its implementations. This is the first step towards making routing data source configurable - other readers will be added in the future commits.
---
 .../helix/manager/zk/GenericZkHelixApiBuilder.java |  10 +-
 .../org/apache/helix/manager/zk/ZKHelixAdmin.java  |   9 +-
 .../apache/helix/rest/server/ServerContext.java    |   6 +-
 .../zookeeper/api/client/RealmAwareZkClient.java   |   4 -
 .../zookeeper/constant/RoutingDataReaderType.java  |  36 ++++
 .../constant/RoutingSystemPropertyKeys.java        |  31 ++++
 .../zookeeper/exception/MultiZkException.java      |  37 ++++
 .../zookeeper/impl/client/DedicatedZkClient.java   |   9 +-
 .../zookeeper/impl/client/FederatedZkClient.java   |   9 +-
 .../zookeeper/impl/client/SharedZkClient.java      |   9 +-
 .../zookeeper/routing/HttpRoutingDataReader.java   | 132 ++++++++++++++
 .../zookeeper/routing/RoutingDataManager.java      | 181 +++++++++++++++++++
 .../helix/zookeeper/routing/RoutingDataReader.java |  45 +++++
 .../zookeeper/util/HttpRoutingDataReader.java      | 197 ---------------------
 ...DataReader.java => TestRoutingDataManager.java} |  13 +-
 15 files changed, 502 insertions(+), 226 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/GenericZkHelixApiBuilder.java b/helix-core/src/main/java/org/apache/helix/manager/zk/GenericZkHelixApiBuilder.java
index c7480c3..840ec8f 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/GenericZkHelixApiBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/GenericZkHelixApiBuilder.java
@@ -26,10 +26,11 @@ import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
 import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
 import org.apache.helix.zookeeper.api.client.HelixZkClient;
 import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
 import org.apache.helix.zookeeper.datamodel.serializer.ZNRecordSerializer;
 import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
 import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -204,9 +205,10 @@ public abstract class GenericZkHelixApiBuilder<B extends GenericZkHelixApiBuilde
       throws IOException, InvalidRoutingDataException {
     boolean isMsdsEndpointSet =
         connectionConfig.getMsdsEndpoint() != null && !connectionConfig.getMsdsEndpoint().isEmpty();
-    MetadataStoreRoutingData routingData = isMsdsEndpointSet ? HttpRoutingDataReader
-        .getMetadataStoreRoutingData(connectionConfig.getMsdsEndpoint())
-        : HttpRoutingDataReader.getMetadataStoreRoutingData();
+    // TODO: Make RoutingDataReaderType configurable
+    MetadataStoreRoutingData routingData = isMsdsEndpointSet ? RoutingDataManager
+        .getMetadataStoreRoutingData(RoutingDataReaderType.HTTP, connectionConfig.getMsdsEndpoint())
+        : RoutingDataManager.getMetadataStoreRoutingData();
     return routingData.getMetadataStoreRealm(connectionConfig.getZkRealmShardingKey());
   }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index 0e123b2..bce8105 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -83,11 +83,12 @@ import org.apache.helix.util.HelixUtil;
 import org.apache.helix.util.RebalanceUtil;
 import org.apache.helix.zookeeper.api.client.HelixZkClient;
 import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
 import org.apache.helix.zookeeper.exception.ZkClientException;
 import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
 import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
 import org.apache.helix.zookeeper.zkclient.exception.ZkException;
 import org.apache.helix.zookeeper.zkclient.exception.ZkNoNodeException;
@@ -967,9 +968,11 @@ public class ZKHelixAdmin implements HelixAdmin {
       String msdsEndpoint = _zkClient.getRealmAwareZkConnectionConfig().getMsdsEndpoint();
       try {
         if (msdsEndpoint == null || msdsEndpoint.isEmpty()) {
-          realmToShardingKeys = HttpRoutingDataReader.getRawRoutingData();
+          realmToShardingKeys = RoutingDataManager.getRawRoutingData();
         } else {
-          realmToShardingKeys = HttpRoutingDataReader.getRawRoutingData(msdsEndpoint);
+          // TODO: Make RoutingDataReaderType configurable
+          realmToShardingKeys =
+              RoutingDataManager.getRawRoutingData(RoutingDataReaderType.HTTP, msdsEndpoint);
         }
       } catch (IOException e) {
         throw new HelixException(
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java b/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
index a9952da..f5d8915 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
@@ -48,7 +48,7 @@ import org.apache.helix.zookeeper.impl.client.FederatedZkClient;
 import org.apache.helix.zookeeper.impl.client.ZkClient;
 import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
 import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.apache.helix.zookeeper.zkclient.IZkChildListener;
 import org.apache.helix.zookeeper.zkclient.IZkDataListener;
 import org.apache.helix.zookeeper.zkclient.IZkStateListener;
@@ -323,8 +323,8 @@ public class ServerContext implements IZkDataListener, IZkChildListener, IZkStat
       LOG.info("ServerContext: Resetting ZK resources due to routing data change! Routing ZK: {}",
           _zkAddr);
       try {
-        // Reset HttpRoutingDataReader's cache
-        HttpRoutingDataReader.reset();
+        // Reset RoutingDataManager's cache
+        RoutingDataManager.reset();
         // All Helix APIs will be closed implicitly because ZkClient is closed
         if (_zkClient != null && !_zkClient.isClosed()) {
           _zkClient.close();
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/api/client/RealmAwareZkClient.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/api/client/RealmAwareZkClient.java
index 288cdfa..b1ecc38 100644
--- a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/api/client/RealmAwareZkClient.java
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/api/client/RealmAwareZkClient.java
@@ -19,14 +19,10 @@ package org.apache.helix.zookeeper.api.client;
  * under the License.
  */
 
-import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
-import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
-import org.apache.helix.zookeeper.exception.ZkClientException;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
 import org.apache.helix.zookeeper.zkclient.IZkChildListener;
 import org.apache.helix.zookeeper.zkclient.IZkDataListener;
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/constant/RoutingDataReaderType.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/constant/RoutingDataReaderType.java
new file mode 100644
index 0000000..85b8194
--- /dev/null
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/constant/RoutingDataReaderType.java
@@ -0,0 +1,36 @@
+package org.apache.helix.zookeeper.constant;
+
+/*
+ * 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.
+ */
+
+public enum RoutingDataReaderType {
+  HTTP("org.apache.helix.zookeeper.routing.HttpRoutingDataReader"),
+  ZK("ZkRoutingDataReader"),
+  HTTP_ZK_FALLBACK("HttpZkFallbackRoutingDataReader");
+
+  private final String className;
+
+  RoutingDataReaderType(String className) {
+    this.className = className;
+  }
+
+  public String getClassName() {
+    return this.className;
+  }
+}
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/constant/RoutingSystemPropertyKeys.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/constant/RoutingSystemPropertyKeys.java
new file mode 100644
index 0000000..23b8ebc
--- /dev/null
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/constant/RoutingSystemPropertyKeys.java
@@ -0,0 +1,31 @@
+package org.apache.helix.zookeeper.constant;
+
+/*
+ * 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.
+ */
+
+/**
+ * This class contains various routing-related system property keys for multi-zk clients.
+ */
+public class RoutingSystemPropertyKeys {
+
+  /**
+   * This static constant is used to refer to which implementation of RoutingDataReader to use.
+   */
+  public static final String ROUTING_DATA_READER_TYPE = "routing.data.reader.type";
+}
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/exception/MultiZkException.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/exception/MultiZkException.java
new file mode 100644
index 0000000..22d6669
--- /dev/null
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/exception/MultiZkException.java
@@ -0,0 +1,37 @@
+package org.apache.helix.zookeeper.exception;
+/*
+ * 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.
+ */
+
+/**
+ * Exception for multi ZK-related failures.
+ */
+public class MultiZkException extends RuntimeException {
+
+  public MultiZkException(String message) {
+    super(message);
+  }
+
+  public MultiZkException(Throwable cause) {
+    super(cause);
+  }
+
+  public MultiZkException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/DedicatedZkClient.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/DedicatedZkClient.java
index 3a50c87..501670c 100644
--- a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/DedicatedZkClient.java
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/DedicatedZkClient.java
@@ -28,7 +28,8 @@ import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
 import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
 import org.apache.helix.zookeeper.api.client.ChildrenSubscribeResult;
 import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
 import org.apache.helix.zookeeper.zkclient.IZkChildListener;
 import org.apache.helix.zookeeper.zkclient.IZkConnection;
@@ -86,9 +87,11 @@ public class DedicatedZkClient implements RealmAwareZkClient {
     // Get the routing data from a static Singleton HttpRoutingDataReader
     String msdsEndpoint = connectionConfig.getMsdsEndpoint();
     if (msdsEndpoint == null || msdsEndpoint.isEmpty()) {
-      _metadataStoreRoutingData = HttpRoutingDataReader.getMetadataStoreRoutingData();
+      _metadataStoreRoutingData = RoutingDataManager.getMetadataStoreRoutingData();
     } else {
-      _metadataStoreRoutingData = HttpRoutingDataReader.getMetadataStoreRoutingData(msdsEndpoint);
+      // TODO: Make RoutingDataReaderType configurable
+      _metadataStoreRoutingData =
+          RoutingDataManager.getMetadataStoreRoutingData(RoutingDataReaderType.HTTP, msdsEndpoint);
     }
 
     _zkRealmShardingKey = connectionConfig.getZkRealmShardingKey();
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/FederatedZkClient.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/FederatedZkClient.java
index f5de9d8..22b9fe5 100644
--- a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/FederatedZkClient.java
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/FederatedZkClient.java
@@ -31,8 +31,9 @@ import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
 import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
 import org.apache.helix.zookeeper.api.client.ChildrenSubscribeResult;
 import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
 import org.apache.helix.zookeeper.impl.factory.DedicatedZkClientFactory;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
 import org.apache.helix.zookeeper.zkclient.IZkChildListener;
 import org.apache.helix.zookeeper.zkclient.IZkDataListener;
@@ -98,9 +99,11 @@ public class FederatedZkClient implements RealmAwareZkClient {
     // Attempt to get MetadataStoreRoutingData
     String msdsEndpoint = connectionConfig.getMsdsEndpoint();
     if (msdsEndpoint == null || msdsEndpoint.isEmpty()) {
-      _metadataStoreRoutingData = HttpRoutingDataReader.getMetadataStoreRoutingData();
+      _metadataStoreRoutingData = RoutingDataManager.getMetadataStoreRoutingData();
     } else {
-      _metadataStoreRoutingData = HttpRoutingDataReader.getMetadataStoreRoutingData(msdsEndpoint);
+      // TODO: Make RoutingDataReaderType configurable
+      _metadataStoreRoutingData =
+          RoutingDataManager.getMetadataStoreRoutingData(RoutingDataReaderType.HTTP, msdsEndpoint);
     }
 
     _isClosed = false;
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
index 4a83712..341731e 100644
--- a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/impl/client/SharedZkClient.java
@@ -29,8 +29,9 @@ import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
 import org.apache.helix.zookeeper.api.client.ChildrenSubscribeResult;
 import org.apache.helix.zookeeper.api.client.HelixZkClient;
 import org.apache.helix.zookeeper.api.client.RealmAwareZkClient;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
 import org.apache.helix.zookeeper.impl.factory.SharedZkClientFactory;
-import org.apache.helix.zookeeper.util.HttpRoutingDataReader;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.apache.helix.zookeeper.zkclient.DataUpdater;
 import org.apache.helix.zookeeper.zkclient.IZkChildListener;
 import org.apache.helix.zookeeper.zkclient.IZkDataListener;
@@ -78,9 +79,11 @@ public class SharedZkClient implements RealmAwareZkClient {
     // Get the routing data from a static Singleton HttpRoutingDataReader
     String msdsEndpoint = connectionConfig.getMsdsEndpoint();
     if (msdsEndpoint == null || msdsEndpoint.isEmpty()) {
-      _metadataStoreRoutingData = HttpRoutingDataReader.getMetadataStoreRoutingData();
+      _metadataStoreRoutingData = RoutingDataManager.getMetadataStoreRoutingData();
     } else {
-      _metadataStoreRoutingData = HttpRoutingDataReader.getMetadataStoreRoutingData(msdsEndpoint);
+      // TODO: Make RoutingDataReaderType configurable
+      _metadataStoreRoutingData =
+          RoutingDataManager.getMetadataStoreRoutingData(RoutingDataReaderType.HTTP, msdsEndpoint);
     }
 
     _zkRealmShardingKey = connectionConfig.getZkRealmShardingKey();
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpRoutingDataReader.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpRoutingDataReader.java
new file mode 100644
index 0000000..4ef9881
--- /dev/null
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/HttpRoutingDataReader.java
@@ -0,0 +1,132 @@
+package org.apache.helix.zookeeper.routing;
+
+/*
+ * 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.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
+import org.apache.helix.msdcommon.datamodel.TrieRoutingData;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.zookeeper.exception.MultiZkException;
+import org.apache.http.HttpEntity;
+import org.apache.http.client.config.RequestConfig;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.DefaultBackoffStrategy;
+import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+
+
+/**
+ * HTTP-based RoutingDataReader that makes an HTTP call to Metadata Store Directory Service (REST)
+ * to fetch routing data.
+ */
+public class HttpRoutingDataReader implements RoutingDataReader {
+  private static final int DEFAULT_HTTP_TIMEOUT_IN_MS = 5000;
+
+  /**
+   * Returns an object form of metadata store routing data.
+   * @param endpoint
+   * @return
+   */
+  @Override
+  public MetadataStoreRoutingData getMetadataStoreRoutingData(String endpoint) {
+    try {
+      return new TrieRoutingData(getRawRoutingData(endpoint));
+    } catch (InvalidRoutingDataException e) {
+      throw new MultiZkException(e);
+    }
+  }
+
+  /**
+   * Returns a map form of metadata store routing data.
+   * The map fields stand for metadata store realm address (key), and a corresponding list of ZK
+   * path sharding keys (key).
+   * @param endpoint
+   * @return
+   */
+  @Override
+  public Map<String, List<String>> getRawRoutingData(String endpoint) {
+    try {
+      String routingDataJson = getAllRoutingData(endpoint);
+      return parseRoutingData(routingDataJson);
+    } catch (IOException e) {
+      throw new MultiZkException(e);
+    }
+  }
+
+  /**
+   * Makes an HTTP call to fetch all routing data.
+   * @return
+   * @throws IOException
+   */
+  private String getAllRoutingData(String msdsEndpoint) throws IOException {
+    // Note that MSDS_ENDPOINT should provide high-availability - it risks becoming a single point
+    // of failure if it's backed by a single IP address/host
+    // Retry count is 3 by default.
+    HttpGet requestAllData = new HttpGet(
+        msdsEndpoint + MetadataStoreRoutingConstants.MSDS_GET_ALL_ROUTING_DATA_ENDPOINT);
+
+    // Define timeout configs
+    RequestConfig config = RequestConfig.custom().setConnectTimeout(DEFAULT_HTTP_TIMEOUT_IN_MS)
+        .setConnectionRequestTimeout(DEFAULT_HTTP_TIMEOUT_IN_MS)
+        .setSocketTimeout(DEFAULT_HTTP_TIMEOUT_IN_MS).build();
+
+    try (CloseableHttpClient httpClient = HttpClients.custom().setDefaultRequestConfig(config)
+        .setConnectionBackoffStrategy(new DefaultBackoffStrategy())
+        .setRetryHandler(new DefaultHttpRequestRetryHandler()).build()) {
+      // Return the JSON because try-resources clause closes the CloseableHttpResponse
+      HttpEntity entity = httpClient.execute(requestAllData).getEntity();
+      if (entity == null) {
+        throw new IOException("Response's entity is null!");
+      }
+      return EntityUtils.toString(entity, "UTF-8");
+    }
+  }
+
+  /**
+   * Returns the raw routing data in a Map< ZkRealm, List of shardingKeys > format.
+   * @param routingDataJson
+   * @return
+   */
+  private Map<String, List<String>> parseRoutingData(String routingDataJson) throws IOException {
+    if (routingDataJson != null) {
+      @SuppressWarnings("unchecked")
+      Map<String, Object> resultMap = new ObjectMapper().readValue(routingDataJson, Map.class);
+      @SuppressWarnings("unchecked")
+      List<Map<String, Object>> routingDataList =
+          (List<Map<String, Object>>) resultMap.get(MetadataStoreRoutingConstants.ROUTING_DATA);
+      @SuppressWarnings("unchecked")
+      Map<String, List<String>> routingData = routingDataList.stream().collect(Collectors.toMap(
+          realmKeyPair -> (String) realmKeyPair
+              .get(MetadataStoreRoutingConstants.SINGLE_METADATA_STORE_REALM),
+          mapEntry -> (List<String>) mapEntry.get(MetadataStoreRoutingConstants.SHARDING_KEYS)));
+      return routingData;
+    }
+    return Collections.emptyMap();
+  }
+}
\ No newline at end of file
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/RoutingDataManager.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/RoutingDataManager.java
new file mode 100644
index 0000000..f8aafb2
--- /dev/null
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/RoutingDataManager.java
@@ -0,0 +1,181 @@
+package org.apache.helix.zookeeper.routing;
+
+/*
+ * 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.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
+import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
+import org.apache.helix.msdcommon.datamodel.TrieRoutingData;
+import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
+import org.apache.helix.zookeeper.constant.RoutingDataReaderType;
+import org.apache.helix.zookeeper.exception.MultiZkException;
+
+
+/**
+ * RoutingDataManager is a static Singleton that
+ * 1. resolves RoutingDataReader based on the system config given
+ * 2. caches routing data
+ * 3. provides public methods for reading routing data from various sources (configurable)
+ */
+public class RoutingDataManager {
+  /** HTTP call to MSDS is used to fetch routing data by default */
+  private static final String DEFAULT_MSDS_ENDPOINT =
+      System.getProperty(MetadataStoreRoutingConstants.MSDS_SERVER_ENDPOINT_KEY);
+
+  /** Double-checked locking requires that the following fields be final (volatile) */
+  // The following map stands for (RoutingDataReaderType_endpoint ID, Raw Routing Data)
+  private static final Map<String, Map<String, List<String>>> _rawRoutingDataMap =
+      new ConcurrentHashMap<>();
+  // The following map stands for (RoutingDataReaderType_endpoint ID, MetadataStoreRoutingData)
+  private static final Map<String, MetadataStoreRoutingData> _metadataStoreRoutingDataMap =
+      new ConcurrentHashMap<>();
+
+  /**
+   * This class is a Singleton.
+   */
+  private RoutingDataManager() {
+  }
+
+  /**
+   * Fetches routing data from the data source via HTTP by querying the MSDS configured in the JVM
+   * config.
+   * @return
+   * @throws IOException
+   */
+  public static Map<String, List<String>> getRawRoutingData() throws IOException {
+    if (DEFAULT_MSDS_ENDPOINT == null || DEFAULT_MSDS_ENDPOINT.isEmpty()) {
+      throw new IllegalStateException(
+          "HttpRoutingDataReader was unable to find a valid MSDS endpoint String in System "
+              + "Properties!");
+    }
+    return getRawRoutingData(RoutingDataReaderType.HTTP, DEFAULT_MSDS_ENDPOINT);
+  }
+
+  /**
+   * Fetches routing data from the data source via HTTP.
+   * @return a mapping from "metadata store realm addresses" to lists of
+   * "metadata store sharding keys", where the sharding keys in a value list all route to
+   * the realm address in the key disallows a meaningful mapping to be returned.
+   * @param routingDataReaderType
+   * @param endpoint
+   */
+  public static Map<String, List<String>> getRawRoutingData(
+      RoutingDataReaderType routingDataReaderType, String endpoint) {
+    String routingDataCacheKey = getRoutingDataCacheKey(routingDataReaderType, endpoint);
+    Map<String, List<String>> rawRoutingData = _rawRoutingDataMap.get(routingDataCacheKey);
+    if (rawRoutingData == null) {
+      synchronized (RoutingDataManager.class) {
+        rawRoutingData = _rawRoutingDataMap.get(routingDataCacheKey);
+        if (rawRoutingData == null) {
+          RoutingDataReader reader = resolveRoutingDataReader(routingDataReaderType);
+          rawRoutingData = reader.getRawRoutingData(endpoint);
+          _rawRoutingDataMap.put(routingDataCacheKey, rawRoutingData);
+        }
+      }
+    }
+    return rawRoutingData;
+  }
+
+  /**
+   * Returns the routing data read from MSDS in a MetadataStoreRoutingData format by querying the
+   * MSDS configured in the JVM config.
+   * @return MetadataStoreRoutingData
+   */
+  public static MetadataStoreRoutingData getMetadataStoreRoutingData()
+      throws InvalidRoutingDataException {
+    if (DEFAULT_MSDS_ENDPOINT == null || DEFAULT_MSDS_ENDPOINT.isEmpty()) {
+      throw new IllegalStateException(
+          "HttpRoutingDataReader was unable to find a valid MSDS endpoint String in System "
+              + "Properties!");
+    }
+    return getMetadataStoreRoutingData(RoutingDataReaderType.HTTP, DEFAULT_MSDS_ENDPOINT);
+  }
+
+  /**
+   * Returns the routing data read from MSDS as a MetadataStoreRoutingData object.
+   * @param routingDataReaderType
+   * @param endpoint
+   * @return
+   * @throws IOException
+   * @throws InvalidRoutingDataException
+   */
+  public static MetadataStoreRoutingData getMetadataStoreRoutingData(
+      RoutingDataReaderType routingDataReaderType, String endpoint)
+      throws InvalidRoutingDataException {
+    String routingDataCacheKey = getRoutingDataCacheKey(routingDataReaderType, endpoint);
+    MetadataStoreRoutingData metadataStoreRoutingData =
+        _metadataStoreRoutingDataMap.get(routingDataCacheKey);
+    if (metadataStoreRoutingData == null) {
+      synchronized (RoutingDataManager.class) {
+        metadataStoreRoutingData = _metadataStoreRoutingDataMap.get(routingDataCacheKey);
+        if (metadataStoreRoutingData == null) {
+          metadataStoreRoutingData =
+              new TrieRoutingData(getRawRoutingData(routingDataReaderType, endpoint));
+          _metadataStoreRoutingDataMap.put(routingDataCacheKey, metadataStoreRoutingData);
+        }
+      }
+    }
+    return metadataStoreRoutingData;
+  }
+
+  /**
+   * Clears the statically-cached routing data and private fields.
+   */
+  public synchronized static void reset() {
+    _rawRoutingDataMap.clear();
+    _metadataStoreRoutingDataMap.clear();
+  }
+
+  /**
+   * Returns an appropriate instance of RoutingDataReader given the type.
+   * @param routingDataReaderType
+   * @return
+   */
+  private static RoutingDataReader resolveRoutingDataReader(
+      RoutingDataReaderType routingDataReaderType) {
+    // RoutingDataReaderType.HTTP by default if not found
+    routingDataReaderType =
+        routingDataReaderType == null ? RoutingDataReaderType.HTTP : routingDataReaderType;
+
+    // Instantiate an instance of routing data reader using the type
+    try {
+      return (RoutingDataReader) Class.forName(routingDataReaderType.getClassName()).newInstance();
+    } catch (ClassNotFoundException | IllegalAccessException | InstantiationException e) {
+      throw new MultiZkException(
+          "RoutingDataManager: failed to instantiate RoutingDataReader! ReaderType className: "
+              + routingDataReaderType.getClassName(), e);
+    }
+  }
+
+  /**
+   * Constructs a key for the cache lookup.
+   * @param routingDataReaderType
+   * @param endpoint
+   * @return
+   */
+  private static String getRoutingDataCacheKey(RoutingDataReaderType routingDataReaderType,
+      String endpoint) {
+    return routingDataReaderType.name() + "_" + endpoint;
+  }
+}
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/RoutingDataReader.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/RoutingDataReader.java
new file mode 100644
index 0000000..9e6c258
--- /dev/null
+++ b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/routing/RoutingDataReader.java
@@ -0,0 +1,45 @@
+package org.apache.helix.zookeeper.routing;
+
+/*
+ * 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.List;
+import java.util.Map;
+
+import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
+
+
+public interface RoutingDataReader {
+
+  /**
+   * Returns an object form of metadata store routing data.
+   * @param endpoint
+   * @return
+   */
+  MetadataStoreRoutingData getMetadataStoreRoutingData(String endpoint);
+
+  /**
+   * Returns a map form of metadata store routing data.
+   * The map fields stand for metadata store realm address (key), and a corresponding list of ZK
+   * path sharding keys (key).
+   * @param endpoint
+   * @return
+   */
+  Map<String, List<String>> getRawRoutingData(String endpoint);
+}
diff --git a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/util/HttpRoutingDataReader.java b/zookeeper-api/src/main/java/org/apache/helix/zookeeper/util/HttpRoutingDataReader.java
deleted file mode 100644
index 6c12046..0000000
--- a/zookeeper-api/src/main/java/org/apache/helix/zookeeper/util/HttpRoutingDataReader.java
+++ /dev/null
@@ -1,197 +0,0 @@
-package org.apache.helix.zookeeper.util;
-
-/*
- * 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.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.helix.msdcommon.constant.MetadataStoreRoutingConstants;
-import org.apache.helix.msdcommon.datamodel.MetadataStoreRoutingData;
-import org.apache.helix.msdcommon.datamodel.TrieRoutingData;
-import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
-import org.apache.http.HttpEntity;
-import org.apache.http.client.config.RequestConfig;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.DefaultBackoffStrategy;
-import org.apache.http.impl.client.DefaultHttpRequestRetryHandler;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.util.EntityUtils;
-
-
-public class HttpRoutingDataReader {
-  private static final String SYSTEM_MSDS_ENDPOINT =
-      System.getProperty(MetadataStoreRoutingConstants.MSDS_SERVER_ENDPOINT_KEY);
-  private static final int HTTP_TIMEOUT_IN_MS = 5000;
-
-  /** Double-checked locking requires that the following fields be volatile */
-  // The following map stands for (MSDS endpoint, Raw Routing Data)
-  private static volatile Map<String, Map<String, List<String>>> _rawRoutingDataMap =
-      new ConcurrentHashMap<>();
-  // The following map stands for (MSDS endpoint, MetadataStoreRoutingData)
-  private static volatile Map<String, MetadataStoreRoutingData> _metadataStoreRoutingDataMap =
-      new ConcurrentHashMap<>();
-
-  /**
-   * This class is a Singleton.
-   */
-  private HttpRoutingDataReader() {
-  }
-
-  /**
-   * Fetches routing data from the data source via HTTP by querying the MSDS configured in the JVM config.
-   * @return
-   * @throws IOException
-   */
-  public static Map<String, List<String>> getRawRoutingData() throws IOException {
-    if (SYSTEM_MSDS_ENDPOINT == null || SYSTEM_MSDS_ENDPOINT.isEmpty()) {
-      throw new IllegalStateException(
-          "HttpRoutingDataReader was unable to find a valid MSDS endpoint String in System Properties!");
-    }
-    return getRawRoutingData(SYSTEM_MSDS_ENDPOINT);
-  }
-
-  /**
-   * Fetches routing data from the data source via HTTP.
-   * @return a mapping from "metadata store realm addresses" to lists of
-   * "metadata store sharding keys", where the sharding keys in a value list all route to
-   * the realm address in the key disallows a meaningful mapping to be returned.
-   * @param msdsEndpoint Metadata Store Directory Store endpoint to query from
-   */
-  public static Map<String, List<String>> getRawRoutingData(String msdsEndpoint)
-      throws IOException {
-    Map<String, List<String>> rawRoutingData = _rawRoutingDataMap.get(msdsEndpoint);
-    if (rawRoutingData == null) {
-      synchronized (HttpRoutingDataReader.class) {
-        rawRoutingData = _rawRoutingDataMap.get(msdsEndpoint);
-        if (rawRoutingData == null) {
-          String routingDataJson = getAllRoutingData(msdsEndpoint);
-          // Update the reference if reading routingData over HTTP is successful
-          rawRoutingData = parseRoutingData(routingDataJson);
-          _rawRoutingDataMap.put(msdsEndpoint, rawRoutingData);
-        }
-      }
-    }
-    return rawRoutingData;
-  }
-
-  /**
-   * Returns the routing data read from MSDS in a MetadataStoreRoutingData format by querying the MSDS configured in the JVM config.
-   * @return MetadataStoreRoutingData
-   * @throws IOException
-   * @throws InvalidRoutingDataException
-   */
-  public static MetadataStoreRoutingData getMetadataStoreRoutingData()
-      throws IOException, InvalidRoutingDataException {
-    if (SYSTEM_MSDS_ENDPOINT == null || SYSTEM_MSDS_ENDPOINT.isEmpty()) {
-      throw new IllegalStateException(
-          "HttpRoutingDataReader was unable to find a valid MSDS endpoint String in System Properties!");
-    }
-    return getMetadataStoreRoutingData(SYSTEM_MSDS_ENDPOINT);
-  }
-
-  /**
-   * Returns the routing data read from MSDS in a MetadataStoreRoutingData format.
-   * @param msdsEndpoint Metadata Store Directory Store endpoint to query from
-   * @return MetadataStoreRoutingData
-   * @throws IOException if there is an issue connecting to MSDS
-   * @throws InvalidRoutingDataException if the raw routing data is not valid
-   */
-  public static MetadataStoreRoutingData getMetadataStoreRoutingData(String msdsEndpoint)
-      throws IOException, InvalidRoutingDataException {
-    MetadataStoreRoutingData metadataStoreRoutingData =
-        _metadataStoreRoutingDataMap.get(msdsEndpoint);
-    if (metadataStoreRoutingData == null) {
-      synchronized (HttpRoutingDataReader.class) {
-        metadataStoreRoutingData = _metadataStoreRoutingDataMap.get(msdsEndpoint);
-        if (metadataStoreRoutingData == null) {
-          metadataStoreRoutingData = new TrieRoutingData(getRawRoutingData(msdsEndpoint));
-          _metadataStoreRoutingDataMap.put(msdsEndpoint, metadataStoreRoutingData);
-        }
-      }
-    }
-    return metadataStoreRoutingData;
-  }
-
-  /**
-   * Clears the statically-cached routing data in HttpRoutingDataReader.
-   */
-  public static void reset() {
-    _rawRoutingDataMap.clear();
-    _metadataStoreRoutingDataMap.clear();
-  }
-
-  /**
-   * Makes an HTTP call to fetch all routing data.
-   * @return
-   * @throws IOException
-   */
-  private static String getAllRoutingData(String msdsEndpoint) throws IOException {
-    // Note that MSDS_ENDPOINT should provide high-availability - it risks becoming a single point
-    // of failure if it's backed by a single IP address/host
-    // Retry count is 3 by default.
-    HttpGet requestAllData = new HttpGet(
-        msdsEndpoint + MetadataStoreRoutingConstants.MSDS_GET_ALL_ROUTING_DATA_ENDPOINT);
-
-    // Define timeout configs
-    RequestConfig config = RequestConfig.custom().setConnectTimeout(HTTP_TIMEOUT_IN_MS)
-        .setConnectionRequestTimeout(HTTP_TIMEOUT_IN_MS).setSocketTimeout(HTTP_TIMEOUT_IN_MS)
-        .build();
-
-    try (CloseableHttpClient httpClient = HttpClients.custom().setDefaultRequestConfig(config)
-        .setConnectionBackoffStrategy(new DefaultBackoffStrategy())
-        .setRetryHandler(new DefaultHttpRequestRetryHandler()).build()) {
-      // Return the JSON because try-resources clause closes the CloseableHttpResponse
-      HttpEntity entity = httpClient.execute(requestAllData).getEntity();
-      if (entity == null) {
-        throw new IOException("Response's entity is null!");
-      }
-      return EntityUtils.toString(entity, "UTF-8");
-    }
-  }
-
-  /**
-   * Returns the raw routing data in a Map< ZkRealm, List of shardingKeys > format.
-   * @param routingDataJson
-   * @return
-   */
-  private static Map<String, List<String>> parseRoutingData(String routingDataJson)
-      throws IOException {
-    if (routingDataJson != null) {
-      @SuppressWarnings("unchecked")
-      Map<String, Object> resultMap = new ObjectMapper().readValue(routingDataJson, Map.class);
-      @SuppressWarnings("unchecked")
-      List<Map<String, Object>> routingDataList =
-          (List<Map<String, Object>>) resultMap.get(MetadataStoreRoutingConstants.ROUTING_DATA);
-      @SuppressWarnings("unchecked")
-      Map<String, List<String>> routingData = routingDataList.stream().collect(Collectors.toMap(
-          realmKeyPair -> (String) realmKeyPair
-              .get(MetadataStoreRoutingConstants.SINGLE_METADATA_STORE_REALM),
-          mapEntry -> (List<String>) mapEntry.get(MetadataStoreRoutingConstants.SHARDING_KEYS)));
-      return routingData;
-    }
-    return Collections.emptyMap();
-  }
-}
\ No newline at end of file
diff --git a/zookeeper-api/src/test/java/org/apache/helix/zookeeper/util/TestHttpRoutingDataReader.java b/zookeeper-api/src/test/java/org/apache/helix/zookeeper/util/TestRoutingDataManager.java
similarity index 90%
rename from zookeeper-api/src/test/java/org/apache/helix/zookeeper/util/TestHttpRoutingDataReader.java
rename to zookeeper-api/src/test/java/org/apache/helix/zookeeper/util/TestRoutingDataManager.java
index 5c52d05..ad16fb3 100644
--- a/zookeeper-api/src/test/java/org/apache/helix/zookeeper/util/TestHttpRoutingDataReader.java
+++ b/zookeeper-api/src/test/java/org/apache/helix/zookeeper/util/TestRoutingDataManager.java
@@ -35,17 +35,18 @@ import org.apache.helix.msdcommon.exception.InvalidRoutingDataException;
 import org.apache.helix.msdcommon.mock.MockMetadataStoreDirectoryServer;
 import org.apache.helix.zookeeper.constant.TestConstants;
 import org.apache.helix.zookeeper.impl.ZkTestBase;
+import org.apache.helix.zookeeper.routing.RoutingDataManager;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 
-public class TestHttpRoutingDataReader extends ZkTestBase {
+public class TestRoutingDataManager extends ZkTestBase {
   private MockMetadataStoreDirectoryServer _msdsServer;
   private final String _host = "localhost";
   private final int _port = 1991;
-  private final String _namespace = "TestHttpRoutingDataReader";
+  private final String _namespace = "TestRoutingDataManager";
 
   @BeforeClass
   public void beforeClass() throws IOException {
@@ -66,14 +67,14 @@ public class TestHttpRoutingDataReader extends ZkTestBase {
 
   @Test
   public void testGetRawRoutingData() throws IOException {
-    Map<String, List<String>> rawRoutingData = HttpRoutingDataReader.getRawRoutingData();
+    Map<String, List<String>> rawRoutingData = RoutingDataManager.getRawRoutingData();
     TestConstants.FAKE_ROUTING_DATA.forEach((realm, keys) -> Assert
         .assertEquals(new HashSet(rawRoutingData.get(realm)), new HashSet(keys)));
   }
 
   @Test(dependsOnMethods = "testGetRawRoutingData")
   public void testGetMetadataStoreRoutingData() throws IOException, InvalidRoutingDataException {
-    MetadataStoreRoutingData data = HttpRoutingDataReader.getMetadataStoreRoutingData();
+    MetadataStoreRoutingData data = RoutingDataManager.getMetadataStoreRoutingData();
     Map<String, String> allMappings = data.getAllMappingUnderPath("/");
     Map<String, Set<String>> groupedMappings = allMappings.entrySet().stream().collect(Collectors
         .groupingBy(Map.Entry::getValue,
@@ -101,7 +102,7 @@ public class TestHttpRoutingDataReader extends ZkTestBase {
 
     // HttpRoutingDataReader should still return old data because it's static
     // Make sure the results don't contain the new realm
-    Map<String, List<String>> rawRoutingData = HttpRoutingDataReader.getRawRoutingData();
+    Map<String, List<String>> rawRoutingData = RoutingDataManager.getRawRoutingData();
     Assert.assertFalse(rawRoutingData.containsKey(newRealm));
 
     // Remove newRealm and check for equality
@@ -110,7 +111,7 @@ public class TestHttpRoutingDataReader extends ZkTestBase {
     TestConstants.FAKE_ROUTING_DATA.forEach((realm, keys) -> Assert
         .assertEquals(new HashSet(rawRoutingData.get(realm)), new HashSet(keys)));
 
-    MetadataStoreRoutingData data = HttpRoutingDataReader.getMetadataStoreRoutingData();
+    MetadataStoreRoutingData data = RoutingDataManager.getMetadataStoreRoutingData();
     Map<String, String> allMappings = data.getAllMappingUnderPath("/");
     Map<String, Set<String>> groupedMappings = allMappings.entrySet().stream().collect(Collectors
         .groupingBy(Map.Entry::getValue,