You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by jx...@apache.org on 2018/01/25 21:49:20 UTC

[29/50] [abbrv] helix git commit: HELIX-661: initial impl of multiple zk support in helix rest

HELIX-661: initial impl of multiple zk support in helix rest


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

Branch: refs/heads/master
Commit: d9052a303b59a3f1f3f347eb5dff28c1bf1c8113
Parents: 019d6f4
Author: hrzhang <hr...@linkedin.com>
Authored: Thu Dec 7 17:10:17 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:20 2018 -0800

----------------------------------------------------------------------
 helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy       |   3 +
 helix-rest/pom.xml                              |   5 +
 .../helix/rest/common/ContextPropertyKeys.java  |   3 +-
 .../helix/rest/common/HelixRestNamespace.java   |  99 +++++++++++++++
 .../helix/rest/common/HelixRestUtils.java       |  60 +++++++++
 .../apache/helix/rest/server/HelixRestMain.java |  43 ++++++-
 .../helix/rest/server/HelixRestServer.java      | 124 ++++++++++++++++---
 .../helix/rest/server/auditlog/AuditLog.java    |  21 +++-
 .../rest/server/filters/AuditLogFilter.java     |  18 ++-
 .../rest/server/resources/AbstractResource.java |   4 +-
 .../helix/rest/server/AbstractTestClass.java    |  45 ++++++-
 .../helix/rest/server/TestClusterAccessor.java  |   2 +
 .../helix/rest/server/TestHelixRestServer.java  |  85 +++++++++++++
 .../rest/server/TestNamespacedAPIAccess.java    |  84 +++++++++++++
 14 files changed, 563 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy b/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
index 5508152..df5262f 100644
--- a/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
+++ b/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
@@ -43,6 +43,9 @@ under the License.
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.14" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)">
         <artifact name="slf4j-log4j12" ext="jar"/>
     </dependency>
+    <dependency org="org.yaml" name="snakeyaml" rev="1.17">
+        <artifact name="snakeyaml" m:classifier="sources" ext="jar"/>
+    </dependency>
 		<dependency org="org.apache.helix" name="helix-core" rev="0.6.10-SNAPSHOT" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
 		<dependency org="org.codehaus.jackson" name="jackson-core-asl" rev="1.8.5" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
 		<dependency org="org.codehaus.jackson" name="jackson-mapper-asl" rev="1.8.5" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/pom.xml
----------------------------------------------------------------------
diff --git a/helix-rest/pom.xml b/helix-rest/pom.xml
index b27f0b9..32b0731 100644
--- a/helix-rest/pom.xml
+++ b/helix-rest/pom.xml
@@ -43,6 +43,11 @@ under the License.
 
   <dependencies>
     <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+      <version>1.17</version>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <version>1.7.25</version>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java b/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
index f2aee51..ce59abc 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
@@ -20,5 +20,6 @@ package org.apache.helix.rest.common;
  */
 
 public enum ContextPropertyKeys {
-  SERVER_CONTEXT
+  SERVER_CONTEXT,
+  NAMESPACE
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
new file mode 100644
index 0000000..5d1c8f3
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
@@ -0,0 +1,99 @@
+package org.apache.helix.rest.common;
+
+/*
+ * 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 class HelixRestNamespace {
+
+  public enum HelixMetadataStoreType {
+    ZOOKEEPER
+  }
+
+  public enum HelixRestNamespaceProperty {
+    NAME,
+    METADATA_STORE_TYPE,
+    METADATA_STORE_ADDRESS,
+    IS_DEFAULT
+  }
+
+  /**
+   * Namespaced object will have path such as /url_prefix/namespaces/{namespace_name}/clusters/...
+   * We are going to have path /url_prefix/clusters/... point to default namespace if there is one
+   */
+  public static final String DEFAULT_NAMESPACE_PATH_SPEC = "/*";
+  public static final String DEFAULT_NAMESPACE_NAME = "default";
+
+  /**
+   * Name of Helix namespace
+   */
+  private String _name;
+
+  /**
+   * Type of a metadata store that belongs to Helix namespace
+   */
+  private HelixMetadataStoreType _metadataStoreType;
+
+  /**
+   * Address of metadata store. Should be informat of
+   * "[ip-address]:[port]" or "[dns-name]:[port]"
+   */
+  private String _metadataStoreAddress;
+
+  /**
+   * Flag indicating whether this namespace is default or not
+   */
+  private boolean _isDefault;
+
+  public HelixRestNamespace(String metadataStoreAddress) throws IllegalArgumentException {
+    this(DEFAULT_NAMESPACE_NAME, HelixMetadataStoreType.ZOOKEEPER, metadataStoreAddress, true);
+  }
+
+  public HelixRestNamespace(String name, HelixMetadataStoreType metadataStoreType, String metadataStoreAddress, boolean isDefault)
+      throws IllegalArgumentException {
+    _name = name;
+    _metadataStoreAddress = metadataStoreAddress;
+    _metadataStoreType = metadataStoreType;
+    _isDefault = isDefault;
+    validate();
+  }
+
+  private void validate() throws IllegalArgumentException {
+    // TODO: add more strict validation for NAME as this will be part of URL
+    if (_name == null || _name.length() == 0) {
+      throw new IllegalArgumentException("Name of namespace not provided");
+    }
+    if (_metadataStoreAddress == null || _metadataStoreAddress.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("Metadata store address \"%s\" is not valid for namespace %s", _metadataStoreAddress, _name));
+    }
+  }
+
+  public boolean isDefault() {
+    return _isDefault;
+  }
+
+  public String getName() {
+    return _name;
+  }
+
+  public String getMetadataStoreAddress() {
+    return _metadataStoreAddress;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
new file mode 100644
index 0000000..6c4a3df
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
@@ -0,0 +1,60 @@
+package org.apache.helix.rest.common;
+
+/*
+ * 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 class HelixRestUtils {
+  /**
+   * Generate servlet path spec for a given namespace.
+   * @param namespace Name of the namespace
+   * @param isDefaultServlet mark this as true to get path spec for the special servlet for default namespace
+   * @return servlet path spec
+   */
+  public static String makeServletPathSpec(String namespace, boolean isDefaultServlet) {
+    return isDefaultServlet ? HelixRestNamespace.DEFAULT_NAMESPACE_PATH_SPEC
+        : String.format("/namespaces/%s/*", namespace);
+  }
+
+  /**
+   * Extract namespace information from servlet path. There are 3 cases:
+   *  1. /namespaces/namespaceName  ->  return namespaceName
+   *  2. /namespaces                ->  return ""
+   *  3. this is special servlet for default namespace  ->  return the reserved name for default namespace
+   * @param servletPath servletPath
+   * @return Namespace name retrieved from servlet spec.
+   */
+  public static String getNamespaceFromServletPath(String servletPath) {
+    if (isDefaultNamespaceServlet(servletPath)) {
+      return HelixRestNamespace.DEFAULT_NAMESPACE_NAME;
+    }
+
+    String namespaceName = servletPath.replace("/namespaces", "");
+    if (namespaceName.isEmpty() || namespaceName.equals("/")) {
+      return "";
+    } else {
+      return namespaceName.replace("/", "");
+    }
+  }
+
+  private static boolean isDefaultNamespaceServlet(String servletPath) {
+    // Special servlet for default namespace has path spec "/*", so servletPath is empty
+    return servletPath == null || servletPath.isEmpty();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
index 7368bbd..ac870d0 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
@@ -19,7 +19,13 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -29,15 +35,18 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.helix.HelixException;
+import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.auditlog.auditloggers.FileBasedAuditLogger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
 
 public class HelixRestMain {
   private static Logger LOG = LoggerFactory.getLogger(HelixRestServer.class);
   private static final String HELP = "help";
   private static final String ZKSERVERADDRESS = "zkSvr";
+  private static final String NAMESPACE_MANIFEST_FILE = "namespace-manifest-file";
   private static final String PORT = "port";
   private static final int DEFAULT_PORT = 8100;
   private static final String URI_PREFIX = "/admin/v2";
@@ -62,6 +71,13 @@ public class HelixRestMain {
     zkServerOption.setRequired(true);
     zkServerOption.setArgName("ZookeeperServerAddress(Required)");
 
+    Option helixRestNamespaceOption = OptionBuilder.withLongOpt(NAMESPACE_MANIFEST_FILE)
+        .withDescription("A yaml file describing helix namespace")
+        .create();
+    helixRestNamespaceOption.setArgs(1);
+    helixRestNamespaceOption.setRequired(false);
+    helixRestNamespaceOption.setArgName("NamespaceManifestFile(Optional)");
+
     Option portOption =
         OptionBuilder.withLongOpt(PORT).withDescription("Provide web service port").create();
     portOption.setArgs(1);
@@ -72,11 +88,12 @@ public class HelixRestMain {
     options.addOption(helpOption);
     options.addOption(zkServerOption);
     options.addOption(portOption);
+    options.addOption(helixRestNamespaceOption);
 
     return options;
   }
 
-  public static void processCommandLineArgs(String[] cliArgs) throws Exception {
+  private static void processCommandLineArgs(String[] cliArgs) throws Exception {
     CommandLineParser cliParser = new GnuParser();
     Options cliOptions = constructCommandLineOptions();
     CommandLine cmd = null;
@@ -90,6 +107,7 @@ public class HelixRestMain {
     }
     int port = DEFAULT_PORT;
     String zkAddr;
+    List<HelixRestNamespace> namespaces = new ArrayList<>();
     if (cmd.hasOption(HELP)) {
       printUsage(cliOptions);
       return;
@@ -98,9 +116,13 @@ public class HelixRestMain {
         port = Integer.parseInt(cmd.getOptionValue(PORT));
       }
       zkAddr = String.valueOf(cmd.getOptionValue(ZKSERVERADDRESS));
+      namespaces.add(new HelixRestNamespace(zkAddr));
+      if (cmd.hasOption(NAMESPACE_MANIFEST_FILE)) {
+        constructNamespaceFromConfigFile(String.valueOf(cmd.getOptionValue(NAMESPACE_MANIFEST_FILE)), namespaces);
+      }
     }
 
-    final HelixRestServer restServer = new HelixRestServer(zkAddr, port, URI_PREFIX,
+    final HelixRestServer restServer = new HelixRestServer(namespaces, port, URI_PREFIX,
         Arrays.<AuditLogger>asList(new FileBasedAuditLogger()));
 
     try {
@@ -113,6 +135,23 @@ public class HelixRestMain {
     }
   }
 
+  private static void constructNamespaceFromConfigFile(String filePath, List<HelixRestNamespace> namespaces)
+      throws IOException {
+    Yaml yaml = new Yaml();
+    @SuppressWarnings("unchecked")
+    ArrayList<Map<String, String>> configs =
+        (ArrayList<Map<String, String>>) yaml.load(new FileInputStream(new File(filePath)));
+    for (Map<String, String> config : configs) {
+      // Currently we don't support adding default namespace through yaml manifest so all
+      // namespaces created here will not be default
+      // TODO: support specifying default namespace from config file
+      namespaces.add(new HelixRestNamespace(config.get(HelixRestNamespace.HelixRestNamespaceProperty.NAME.name()),
+          HelixRestNamespace.HelixMetadataStoreType.valueOf(
+              config.get(HelixRestNamespace.HelixRestNamespaceProperty.METADATA_STORE_TYPE.name())),
+          config.get(HelixRestNamespace.HelixRestNamespaceProperty.METADATA_STORE_ADDRESS.name()), false));
+    }
+  }
+
   /**
    * @param args
    * @throws Exception

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
index 20f5b1b..3737308 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
@@ -19,16 +19,26 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import org.apache.helix.HelixException;
 import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.apache.helix.rest.common.HelixRestUtils;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.filters.AuditLogFilter;
 import org.apache.helix.rest.server.filters.CORSFilter;
 import org.apache.helix.rest.server.resources.AbstractResource;
 import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.*;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
@@ -37,27 +47,87 @@ import org.glassfish.jersey.servlet.ServletContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HelixRestServer extends ResourceConfig {
+public class HelixRestServer {
   private static Logger LOG = LoggerFactory.getLogger(HelixRestServer.class);
 
   private int _port;
   private String _urlPrefix;
   private Server _server;
-  private ServerContext _serverContext;
+  private ServletContextHandler _servletContextHandler;
+  private List<AuditLogger> _auditLoggers;
+
+  // Key is name of namespace, value of the resource config of that namespace
+  private Map<String, ResourceConfig> _resourceConfigMap;
+
+  // In additional to regular servlets serving namespaced API endpoints, We have a default servlet
+  // serving un-namespaced API (/admin/v2/clusters/...) for default namespace as well. We use this
+  // literal as a key in _resourceConfigMap to keep records for default servlet.
+  // TODO: try to find a way to serve 2 sets of endpoints of default namespace in 1 servlet
+  private static final String DEFAULT_SERVLET_KEY = "DefaultServlet";
+
+  public HelixRestServer(String zkAddr, int port, String urlPrefix) {
+    this(zkAddr, port, urlPrefix, Collections.<AuditLogger>emptyList());
+  }
 
   public HelixRestServer(String zkAddr, int port, String urlPrefix, List<AuditLogger> auditLoggers) {
+    // Create default namespace using zkAddr
+    ArrayList<HelixRestNamespace> namespaces = new ArrayList<>();
+    namespaces.add(new HelixRestNamespace(HelixRestNamespace.DEFAULT_NAMESPACE_NAME,
+        HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, zkAddr, true));
+    init(namespaces, port, urlPrefix, auditLoggers);
+  }
+
+  public HelixRestServer(List<HelixRestNamespace> namespaces, int port, String urlPrefix, List<AuditLogger> auditLoggers) {
+    init(namespaces, port, urlPrefix, auditLoggers);
+  }
+
+  private void init(List<HelixRestNamespace> namespaces, int port, String urlPrefix,
+      List<AuditLogger> auditLoggers) {
+    if (namespaces.size() == 0) {
+      throw new IllegalArgumentException(
+          "No namespace specified! Please provide ZOOKEEPER address or namespace manifest.");
+    }
     _port = port;
     _urlPrefix = urlPrefix;
     _server = new Server(_port);
+    _auditLoggers = auditLoggers;
+    _resourceConfigMap = new HashMap<>();
+    _servletContextHandler = new ServletContextHandler(_server, _urlPrefix);
 
-    packages(AbstractResource.class.getPackage().getName());
-
-    _serverContext = new ServerContext(zkAddr);
-    property(ContextPropertyKeys.SERVER_CONTEXT.name(), _serverContext);
-
-    register(new CORSFilter());
-    register(new AuditLogFilter(auditLoggers));
+    // Initialize all namespaces
+    try {
+      for (HelixRestNamespace namespace : namespaces) {
+        LOG.info("Initializing namespace " + namespace.getName());
+        if (_resourceConfigMap.containsKey(namespace.getName())) {
+          throw new IllegalArgumentException(String.format("Duplicated namespace name \"%s\"", namespace.getName()));
+        }
+
+        // Create resource and context for namespaced servlet
+        _resourceConfigMap.put(namespace.getName(),
+            makeResourceConfig(namespace, AbstractResource.class.getPackage().getName()));
+        LOG.info("Initializing servlet for namespace " + namespace.getName());
+        initServlet(_resourceConfigMap.get(namespace.getName()),
+            HelixRestUtils.makeServletPathSpec(namespace.getName(), false));
+
+        // Create special resource and context for default namespace servlet
+        if (namespace.isDefault()) {
+          if (_resourceConfigMap.containsKey(DEFAULT_SERVLET_KEY)) {
+            throw new IllegalArgumentException("More than 1 default namespaces are provided");
+          }
+          LOG.info("Creating special servlet for default namespace");
+          _resourceConfigMap.put(DEFAULT_SERVLET_KEY,
+              makeResourceConfig(namespace, AbstractResource.class.getPackage().getName()));
+          initServlet(_resourceConfigMap.get(DEFAULT_SERVLET_KEY),
+              HelixRestUtils.makeServletPathSpec(namespace.getName(), true));
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Failed to initialize helix rest server. Tearing down.");
+      cleanupResourceConfigs();
+      throw e;
+    }
 
+    // Start special servlet for serving namespaces
     Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
       @Override public void run() {
         shutdown();
@@ -65,15 +135,21 @@ public class HelixRestServer extends ResourceConfig {
     }));
   }
 
-  public HelixRestServer(String zkAddr, int port, String urlPrefix) {
-    this(zkAddr, port, urlPrefix, Collections.<AuditLogger>emptyList());
+  private ResourceConfig makeResourceConfig(HelixRestNamespace ns, String... packages) {
+    ResourceConfig cfg = new ResourceConfig();
+    cfg.packages(packages)
+        .property(ContextPropertyKeys.SERVER_CONTEXT.name(), new ServerContext(ns.getMetadataStoreAddress()))
+        .register(new CORSFilter())
+        .register(new AuditLogFilter(_auditLoggers));
+    return cfg;
   }
 
-  public void start() throws HelixException, InterruptedException {
-    ServletHolder servlet = new ServletHolder(new ServletContainer(this));
-    ServletContextHandler contextHandler = new ServletContextHandler(_server, _urlPrefix);
-    contextHandler.addServlet(servlet, "/*");
+  private void initServlet(ResourceConfig cfg, String servletPathSpec) {
+    ServletHolder servlet = new ServletHolder(new ServletContainer(cfg));
+    _servletContextHandler.addServlet(servlet, servletPathSpec);
+  }
 
+  public void start() throws HelixException, InterruptedException {
     try {
       _server.start();
     } catch (Exception ex) {
@@ -103,9 +179,19 @@ public class HelixRestServer extends ResourceConfig {
         LOG.error("Failed to stop Helix rest server, " + ex);
       }
     }
-    ServerContext serverContext =
-        (ServerContext) getProperty(ContextPropertyKeys.SERVER_CONTEXT.name());
-    serverContext.close();
+    cleanupResourceConfigs();
+  }
+
+  private void cleanupResourceConfigs() {
+    for (Map.Entry<String, ResourceConfig> e : _resourceConfigMap.entrySet()) {
+      ServerContext ctx = (ServerContext) e.getValue().getProperty(ContextPropertyKeys.SERVER_CONTEXT.name());
+      if (ctx == null) {
+        LOG.warn("Server context for servlet " + e.getKey() + " is null.");
+      } else {
+        LOG.info("Closing context for servlet " + e.getKey());
+        ctx.close();
+      }
+    }
   }
 
   public void setupSslServer(int port, SslContextFactory sslContextFactory) {

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java b/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
index c857e6d..2afa4ba 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
@@ -35,6 +35,7 @@ public class AuditLog {
   private Principal _principal;
   private String _clientIP;
   private String _clientHostPort;
+  private String _namespace;
   private String _requestPath;
   private String _httpMethod;
   private List<String> _requestHeaders;
@@ -46,7 +47,7 @@ public class AuditLog {
   private String _additionalInfo;
 
   public AuditLog(Date startTime, Date completeTime, Principal principal, String clientIP,
-      String clientHostPort, String requestPath, String httpMethod, List<String> requestHeaders,
+      String clientHostPort, String namespace, String requestPath, String httpMethod, List<String> requestHeaders,
       String requestEntity, int responseCode, String responseEntity,
       String additionalInfo, List<Exception> exceptions) {
     _startTime = startTime;
@@ -54,6 +55,7 @@ public class AuditLog {
     _principal = principal;
     _clientIP = clientIP;
     _clientHostPort = clientHostPort;
+    _namespace = namespace;
     _requestPath = requestPath;
     _httpMethod = httpMethod;
     _requestHeaders = requestHeaders;
@@ -72,6 +74,7 @@ public class AuditLog {
         ", _principal=" + _principal +
         ", _clientIP='" + _clientIP + '\'' +
         ", _clientHostPort='" + _clientHostPort + '\'' +
+        ", _namespace='" + _namespace + '\'' +
         ", _requestPath='" + _requestPath + '\'' +
         ", _httpMethod='" + _httpMethod + '\'' +
         ", _requestHeaders=" + _requestHeaders +
@@ -103,6 +106,10 @@ public class AuditLog {
     return _clientHostPort;
   }
 
+  public String getNamespace() {
+    return _namespace;
+  }
+
   public String getRequestPath() {
     return _requestPath;
   }
@@ -141,6 +148,7 @@ public class AuditLog {
     private Principal _principal;
     private String _clientIP;
     private String _clientHostPort;
+    private String _namespace;
     private String _requestPath;
     private String _httpMethod;
     private List<String> _requestHeaders;
@@ -196,6 +204,15 @@ public class AuditLog {
       return this;
     }
 
+    public String getNamespace() {
+      return _namespace;
+    }
+
+    public Builder namespace(String namespace) {
+      _namespace = namespace;
+      return this;
+    }
+
     public String getRequestPath() {
       return _requestPath;
     }
@@ -278,7 +295,7 @@ public class AuditLog {
 
     public AuditLog build() {
       return new AuditLog(_startTime, _completeTime, _principal, _clientIP, _clientHostPort,
-          _requestPath, _httpMethod, _requestHeaders, _requestEntity, _responseCode,
+          _namespace, _requestPath, _httpMethod, _requestHeaders, _requestEntity, _responseCode,
           _responseEntity, _additionalInfo, _exceptions);
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java b/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
index 235b0bb..646e3d3 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
@@ -37,6 +37,7 @@ import javax.ws.rs.container.PreMatching;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.ext.Provider;
+import org.apache.helix.rest.common.HelixRestUtils;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.slf4j.Logger;
@@ -59,9 +60,14 @@ public class AuditLogFilter implements ContainerRequestFilter, ContainerResponse
   @Override
   public void filter(ContainerRequestContext request) throws IOException {
     AuditLog.Builder auditLogBuilder = new AuditLog.Builder();
-    auditLogBuilder.requestPath(request.getUriInfo().getPath()).httpMethod(request.getMethod())
-        .startTime(new Date()).requestHeaders(getHeaders(request.getHeaders()))
-        .principal(_servletRequest.getUserPrincipal()).clientIP(_servletRequest.getRemoteAddr())
+
+    auditLogBuilder.namespace(getNamespace())
+        .requestPath(request.getUriInfo().getPath())
+        .httpMethod(request.getMethod())
+        .startTime(new Date())
+        .requestHeaders(getHeaders(request.getHeaders()))
+        .principal(_servletRequest.getUserPrincipal())
+        .clientIP(_servletRequest.getRemoteAddr())
         .clientHostPort(_servletRequest.getRemoteHost() + ":" + _servletRequest.getRemotePort());
 
     String entity = getEntity(request.getEntityStream());
@@ -115,4 +121,10 @@ public class AuditLogFilter implements ContainerRequestFilter, ContainerResponse
     }
     return null;
   }
+
+  private String getNamespace() {
+    String servletPath = _servletRequest.getServletPath();
+    return HelixRestUtils.getNamespaceFromServletPath(servletPath);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
index 50f6f08..70f713a 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
@@ -41,10 +41,10 @@ import org.apache.helix.rest.server.ServerContext;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Produces({MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN})
 @Consumes({MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN})

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
index 29b198e..8b86c17 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
@@ -33,7 +33,6 @@ import javax.ws.rs.client.Entity;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.Application;
 import javax.ws.rs.core.Response;
-
 import org.I0Itec.zkclient.ZkServer;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
@@ -48,6 +47,7 @@ import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.filters.AuditLogFilter;
@@ -88,6 +88,13 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
   protected static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
   protected static boolean _init = false;
 
+  // For testing namespaced access
+  protected static ZkServer _zkServerTestNS;
+  protected static final String _zkAddrTestNS = "localhost:2124";
+  protected static final String TEST_NAMESPACE = "test-namespace";
+  protected static ZkClient _gZkClientTestNS;
+  protected static BaseDataAccessor<ZNRecord> _baseAccessorTestNS;
+
   protected static Set<String> _clusters;
   protected static String _superCluster = "superCluster";
   protected static Map<String, Set<String>> _instancesMap = new HashMap<>();
@@ -123,6 +130,12 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
         Assert.assertTrue(_zkServer != null);
         ZKClientPool.reset();
       }
+
+      if (_zkServerTestNS == null) {
+        _zkServerTestNS = TestHelper.startZkServer(_zkAddrTestNS);
+        Assert.assertTrue(_zkServerTestNS != null);
+        ZKClientPool.reset();
+      }
     } catch (Exception e) {
       Assert.assertTrue(false, String.format("Failed to start ZK server: %s", e.toString()));
     }
@@ -157,15 +170,21 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
 
           @Override
           public void start() {
+            // Create namespace manifest map
+            List<HelixRestNamespace> namespaces = new ArrayList<>();
+            // Add test namespace
+            namespaces.add(new HelixRestNamespace(TEST_NAMESPACE, HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER,
+                _zkAddrTestNS, false));
+            // Add default namesapce
+            namespaces.add(new HelixRestNamespace(ZK_ADDR));
             try {
-              _helixRestServer = new HelixRestServer(ZK_ADDR, baseUri.getPort(), baseUri.getPath(),
+              _helixRestServer = new HelixRestServer(namespaces, baseUri.getPort(), baseUri.getPath(),
                   Arrays.<AuditLogger>asList(_auditLogger));
               _helixRestServer.start();
             } catch (Exception ex) {
               throw new TestContainerException(ex);
             }
           }
-
           @Override
           public void stop() {
             _helixRestServer.shutdown();
@@ -184,9 +203,12 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
 
       _gZkClient = new ZkClient(ZK_ADDR, ZkClient.DEFAULT_CONNECTION_TIMEOUT,
           ZkClient.DEFAULT_SESSION_TIMEOUT, new ZNRecordSerializer());
+      _gZkClientTestNS = new ZkClient(_zkAddrTestNS, ZkClient.DEFAULT_CONNECTION_TIMEOUT, ZkClient.DEFAULT_SESSION_TIMEOUT,
+          new ZNRecordSerializer());
       _gSetupTool = new ClusterSetup(_gZkClient);
       _configAccessor = new ConfigAccessor(_gZkClient);
       _baseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+      _baseAccessorTestNS = new ZkBaseDataAccessor<>(_gZkClientTestNS);
 
       // wait for the web service to start
       Thread.sleep(100);
@@ -208,6 +230,15 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
       TestHelper.stopZkServer(_zkServer);
       _zkServer = null;
     }
+
+    if (_gZkClientTestNS != null) {
+      _gZkClientTestNS.close();
+      _gZkClientTestNS = null;
+    }
+    if (_zkServerTestNS != null) {
+      TestHelper.stopZkServer(_zkServerTestNS);
+      _zkServerTestNS = null;
+    }
   }
 
   protected void setup() throws Exception {
@@ -342,7 +373,13 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
   protected String get(String uri, int expectedReturnStatus, boolean expectBodyReturned) {
     final Response response = target(uri).request().get();
     Assert.assertEquals(response.getStatus(), expectedReturnStatus);
-    Assert.assertEquals(response.getMediaType().getType(), "application");
+
+    // NOT_FOUND will throw text based html
+    if (expectedReturnStatus != Response.Status.NOT_FOUND.getStatusCode()) {
+      Assert.assertEquals(response.getMediaType().getType(), "application");
+    } else {
+      Assert.assertEquals(response.getMediaType().getType(), "text");
+    }
 
     String body = response.readEntity(String.class);
     if (expectBodyReturned) {

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
index b48c01b..94c5f63 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
@@ -39,6 +39,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.resources.AbstractResource.Command;
 import org.apache.helix.rest.server.resources.ClusterAccessor;
@@ -315,6 +316,7 @@ public class TestClusterAccessor extends AbstractTestClass {
     Assert.assertNotNull(auditLog.getClientHostPort());
     Assert.assertNotNull(auditLog.getCompleteTime());
     Assert.assertNotNull(auditLog.getStartTime());
+    Assert.assertEquals(auditLog.getNamespace(), HelixRestNamespace.DEFAULT_NAMESPACE_NAME);
     Assert.assertEquals(auditLog.getRequestPath(), requestPath);
     Assert.assertEquals(auditLog.getResponseCode(), statusCode);
     Assert.assertEquals(auditLog.getResponseEntity(), responseEntity);

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
new file mode 100644
index 0000000..e213bd3
--- /dev/null
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
@@ -0,0 +1,85 @@
+package org.apache.helix.rest.server;
+
+/*
+ * 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.Collections;
+import java.util.List;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.apache.helix.rest.server.auditlog.AuditLogger;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestHelixRestServer extends AbstractTestClass {
+  @Test
+  public void testInvalidHelixRestServerInitialization() {
+    // Namespace manifests has invalid metadata store type should generate failure
+    try {
+      List<HelixRestNamespace> invalidManifest1 = new ArrayList<>();
+      invalidManifest1.add(
+          new HelixRestNamespace("test1", HelixRestNamespace.HelixMetadataStoreType.valueOf("InvalidMetadataStore"),
+              ZK_ADDR, false));
+      HelixRestServer svr = new HelixRestServer(invalidManifest1, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest1 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+
+    // Namespace manifests has invalid namespace name shall generate failure
+    try {
+      List<HelixRestNamespace> invalidManifest2 = new ArrayList<>();
+      invalidManifest2.add(
+          new HelixRestNamespace("", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
+      HelixRestServer svr = new HelixRestServer(invalidManifest2, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest2 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+
+    // Duplicated namespace shall cause exception
+    try {
+      List<HelixRestNamespace> invalidManifest3 = new ArrayList<>();
+      invalidManifest3.add(
+          new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR,
+              true));
+      invalidManifest3.add(
+          new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR,
+              false));
+      HelixRestServer svr = new HelixRestServer(invalidManifest3, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest3 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+
+    // More than 1 default namespace shall cause failure
+    try {
+      List<HelixRestNamespace> invalidManifest4 = new ArrayList<>();
+      invalidManifest4.add(
+          new HelixRestNamespace("test4-1", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
+      invalidManifest4.add(
+          new HelixRestNamespace("test4-2", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
+      HelixRestServer svr = new HelixRestServer(invalidManifest4, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest4 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
new file mode 100644
index 0000000..e6f036d
--- /dev/null
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
@@ -0,0 +1,84 @@
+package org.apache.helix.rest.server;
+
+/*
+ * 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 com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestNamespacedAPIAccess extends AbstractTestClass {
+  @Test
+  public void testDefaultNamespaceCompatibility() {
+    String testClusterName1 = "testClusterForDefaultNamespaceCompatibility1";
+    String testClusterName2 = "testClusterForDefaultNamespaceCompatibility2";
+
+    // Create from namespaced API and ensure we can access it from old apis, and vice-versa
+    // Assume other api end points will behave the same way
+    put(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName1), null,
+        Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.CREATED.getStatusCode());
+    get(String.format("/clusters/%s", testClusterName1), Response.Status.OK.getStatusCode(), false);
+
+    put(String.format("/clusters/%s", testClusterName2), null, Entity.entity("", MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+    get(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName2),
+        Response.Status.OK.getStatusCode(), false);
+  }
+
+
+  @Test
+  public void testNamespacedCRUD() throws IOException {
+    String testClusterName = "testClusterForNamespacedCRUD";
+
+    // Create cluster in test namespace and verify it's only appears in test namespace
+    put(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName), null,
+        Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.CREATED.getStatusCode());
+    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+        Response.Status.OK.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), Response.Status.NOT_FOUND.getStatusCode(), false);
+
+    // Create cluster with same name in different namespacces
+    put(String.format("/clusters/%s", testClusterName), null, Entity.entity("", MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+    get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
+
+    // Modify cluster in default namespace
+    post(String.format("/clusters/%s", testClusterName), ImmutableMap.of("command", "disable"),
+        Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.OK.getStatusCode());
+
+    // Verify the cluster in default namespace is modified, while the one in test namespace is not.
+    PropertyKey.Builder keyBuilder = new PropertyKey.Builder(testClusterName);
+    Assert.assertTrue(_baseAccessor.exists(keyBuilder.pause().getPath(), 0));
+    Assert.assertFalse(_baseAccessorTestNS.exists(keyBuilder.pause().getPath(), 0));
+
+    // Verify that deleting cluster in one namespace will not affect the other
+    delete(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+        Response.Status.OK.getStatusCode());
+    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+        Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
+  }
+
+}