You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/09/22 04:03:49 UTC

[12/12] hbase git commit: HBASE-14147 Add namespace CRUD functionality to REST

HBASE-14147 Add namespace CRUD functionality to REST

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/master
Commit: 783e20e1a7ca2c796b8d9c9e938e5c5779aa30c7
Parents: 8765ffb
Author: Matt Warhaftig <mw...@gmail.com>
Authored: Mon Aug 10 01:23:58 2015 -0400
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Sep 21 19:03:24 2015 -0700

----------------------------------------------------------------------
 hbase-rest/pom.xml                              |    2 +
 .../hbase/rest/NamespacesInstanceResource.java  |  325 ++++
 .../hadoop/hbase/rest/NamespacesResource.java   |   90 ++
 .../apache/hadoop/hbase/rest/RootResource.java  |    5 +
 .../rest/model/NamespacesInstanceModel.java     |  168 +++
 .../hbase/rest/model/NamespacesModel.java       |  116 ++
 .../generated/NamespacePropertiesMessage.java   | 1394 ++++++++++++++++++
 .../protobuf/generated/NamespacesMessage.java   |  547 +++++++
 .../rest/provider/JAXBContextResolver.java      |    4 +
 .../org/apache/hadoop/hbase/rest/XMLSchema.xsd  |   28 +
 .../protobuf/NamespacePropertiesMessage.proto   |   26 +
 .../hbase/rest/protobuf/NamespacesMessage.proto |   22 +
 .../rest/TestNamespacesInstanceResource.java    |  441 ++++++
 .../hbase/rest/TestNamespacesResource.java      |  204 +++
 .../rest/model/TestNamespacesInstanceModel.java |   98 ++
 .../hbase/rest/model/TestNamespacesModel.java   |   86 ++
 16 files changed, 3556 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 2c3ec3f..3443afc 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -366,6 +366,8 @@
                       <include>CellMessage.proto</include>
                       <include>CellSetMessage.proto</include>
                       <include>ColumnSchemaMessage.proto</include>
+                      <include>NamespacesMessage.proto</include>
+                      <include>NamespacePropertiesMessage.proto</include>
                       <include>ScannerMessage.proto</include>
                       <include>StorageClusterStatusMessage.proto</include>
                       <include>TableInfoMessage.proto</include>

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
new file mode 100644
index 0000000..8f64738
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
@@ -0,0 +1,325 @@
+/*
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.rest;
+
+import java.io.IOException;
+
+import javax.servlet.ServletContext;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.model.TableListModel;
+import org.apache.hadoop.hbase.rest.model.TableModel;
+
+/**
+ * Implements the following REST end points:
+ * <p>
+ * <tt>/namespaces/{namespace} GET: get namespace properties.</tt>
+ * <tt>/namespaces/{namespace} POST: create namespace.</tt>
+ * <tt>/namespaces/{namespace} PUT: alter namespace.</tt>
+ * <tt>/namespaces/{namespace} DELETE: drop namespace.</tt>
+ * <tt>/namespaces/{namespace}/tables GET: list namespace's tables.</tt>
+ * <p>
+ */
+@InterfaceAudience.Private
+public class NamespacesInstanceResource extends ResourceBase {
+
+  private static final Log LOG = LogFactory.getLog(NamespacesInstanceResource.class);
+  String namespace;
+  boolean queryTables = false;
+
+  /**
+   * Constructor for standard NamespaceInstanceResource.
+   * @throws IOException
+   */
+  public NamespacesInstanceResource(String namespace) throws IOException {
+    this(namespace, false);
+  }
+
+  /**
+   * Constructor for querying namespace table list via NamespaceInstanceResource.
+   * @throws IOException
+   */
+  public NamespacesInstanceResource(String namespace, boolean queryTables) throws IOException {
+    super();
+    this.namespace = namespace;
+    this.queryTables = queryTables;
+  }
+
+  /**
+   * Build a response for GET namespace description or GET list of namespace tables.
+   * @param context servlet context
+   * @param uriInfo (JAX-RS context variable) request URL
+   * @return A response containing NamespacesInstanceModel for a namespace descriptions and
+   * TableListModel for a list of namespace tables.
+   */
+  @GET
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
+  public Response get(final @Context ServletContext context,
+      final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("GET " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+
+    // Respond to list of namespace tables requests.
+    if(queryTables){
+      TableListModel tableModel = new TableListModel();
+      try{
+        HTableDescriptor[] tables = servlet.getAdmin().listTableDescriptorsByNamespace(namespace);
+        for(int i = 0; i < tables.length; i++){
+          tableModel.add(new TableModel(tables[i].getTableName().getQualifierAsString()));
+        }
+
+        servlet.getMetrics().incrementSucessfulGetRequests(1);
+        return Response.ok(tableModel).build();
+      }catch(IOException e) {
+        servlet.getMetrics().incrementFailedGetRequests(1);
+        throw new RuntimeException("Cannot retrieve table list for '" + namespace + "'.");
+      }
+    }
+
+    // Respond to namespace description requests.
+    try {
+      NamespacesInstanceModel rowModel =
+          new NamespacesInstanceModel(servlet.getAdmin(), namespace);
+      servlet.getMetrics().incrementSucessfulGetRequests(1);
+      return Response.ok(rowModel).build();
+    } catch (IOException e) {
+      servlet.getMetrics().incrementFailedGetRequests(1);
+      throw new RuntimeException("Cannot retrieve info for '" + namespace + "'.");
+    }
+  }
+
+  /**
+   * Build a response for PUT alter namespace with properties specified.
+   * @param model properties used for alter.
+   * @param uriInfo (JAX-RS context variable) request URL
+   * @return response code.
+   */
+  @PUT
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
+  public Response put(final NamespacesInstanceModel model, final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("PUT " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+    return processUpdate(model, true, uriInfo);
+  }
+
+  /**
+   * Build a response for PUT alter namespace with no properties specified.
+   * @param message value not used.
+   * @param headers value not used.
+   * @return response code.
+   */
+  @PUT
+  public Response putNoBody(final byte[] message,
+      final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("PUT " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+    try{
+      NamespacesInstanceModel model = new NamespacesInstanceModel(namespace);
+      return processUpdate(model, true, uriInfo);
+    }catch(IOException ioe){
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      throw new RuntimeException("Cannot retrieve info for '" + namespace + "'.");
+    }
+  }
+
+  /**
+   * Build a response for POST create namespace with properties specified.
+   * @param model properties used for create.
+   * @param uriInfo (JAX-RS context variable) request URL
+   * @return response code.
+   */
+  @POST
+  @Consumes({MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
+  public Response post(final NamespacesInstanceModel model,
+      final @Context UriInfo uriInfo) {
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("POST " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+    return processUpdate(model, false, uriInfo);
+  }
+
+  /**
+   * Build a response for POST create namespace with no properties specified.
+   * @param message value not used.
+   * @param headers value not used.
+   * @return response code.
+   */
+  @POST
+  public Response postNoBody(final byte[] message,
+      final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("POST " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+    try{
+      NamespacesInstanceModel model = new NamespacesInstanceModel(namespace);
+      return processUpdate(model, false, uriInfo);
+    }catch(IOException ioe){
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      throw new RuntimeException("Cannot retrieve info for '" + namespace + "'.");
+    }
+  }
+
+  // Check that POST or PUT is valid and then update namespace.
+  private Response processUpdate(final NamespacesInstanceModel model, final boolean updateExisting,
+      final UriInfo uriInfo) {
+    if (servlet.isReadOnly()) {
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      return Response.status(Response.Status.FORBIDDEN).type(MIMETYPE_TEXT)
+          .entity("Forbidden" + CRLF).build();
+    }
+
+    Admin admin = null;
+    boolean namespaceExists = false;
+    try {
+      admin = servlet.getAdmin();
+      namespaceExists = doesNamespaceExist(admin, namespace);
+    }catch (IOException e) {
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      return processException(e);
+    }
+
+    // Do not allow creation if namespace already exists.
+    if(!updateExisting && namespaceExists){
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      return Response.status(Response.Status.FORBIDDEN).type(MIMETYPE_TEXT).
+          entity("Namespace '" + namespace + "' already exists.  Use REST PUT " +
+          "to alter the existing namespace.").build();
+    }
+
+    // Do not allow altering if namespace does not exist.
+    if (updateExisting && !namespaceExists){
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      return Response.status(Response.Status.FORBIDDEN).type(MIMETYPE_TEXT).
+          entity("Namespace '" + namespace + "' does not exist. Use " +
+          "REST POST to create the namespace.").build();
+    }
+
+    return createOrUpdate(model, uriInfo, admin, updateExisting);
+  }
+
+  // Do the actual namespace create or alter.
+  private Response createOrUpdate(final NamespacesInstanceModel model, final UriInfo uriInfo,
+      final Admin admin, final boolean updateExisting) {
+    NamespaceDescriptor.Builder builder = NamespaceDescriptor.create(namespace);
+    builder.addConfiguration(model.getProperties());
+    if(model.getProperties().size() > 0){
+      builder.addConfiguration(model.getProperties());
+    }
+    NamespaceDescriptor nsd = builder.build();
+
+    try{
+      if(updateExisting){
+        admin.modifyNamespace(nsd);
+      }else{
+        admin.createNamespace(nsd);
+      }
+    }catch (IOException e) {
+      servlet.getMetrics().incrementFailedPutRequests(1);
+      return processException(e);
+    }
+
+    servlet.getMetrics().incrementSucessfulPutRequests(1);
+    return Response.created(uriInfo.getAbsolutePath()).build();
+  }
+
+  private boolean doesNamespaceExist(Admin admin, String namespaceName) throws IOException{
+    NamespaceDescriptor[] nd = admin.listNamespaceDescriptors();
+    for(int i = 0; i < nd.length; i++){
+      if(nd[i].getName().equals(namespaceName)){
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Build a response for DELETE delete namespace.
+   * @param message value not used.
+   * @param headers value not used.
+   * @return response code.
+   */
+  @DELETE
+  public Response deleteNoBody(final byte[] message,
+      final @Context UriInfo uriInfo, final @Context HttpHeaders headers) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("DELETE " + uriInfo.getAbsolutePath());
+    }
+    if (servlet.isReadOnly()) {
+      servlet.getMetrics().incrementFailedDeleteRequests(1);
+      return Response.status(Response.Status.FORBIDDEN).type(MIMETYPE_TEXT)
+          .entity("Forbidden" + CRLF).build();
+    }
+
+    try{
+      Admin admin = servlet.getAdmin();
+      if (!doesNamespaceExist(admin, namespace)){
+        return Response.status(Response.Status.NOT_FOUND).type(MIMETYPE_TEXT).
+            entity("Namespace '" + namespace + "' does not exists.  Cannot " +
+            "drop namespace.").build();
+      }
+
+      admin.deleteNamespace(namespace);
+      servlet.getMetrics().incrementSucessfulDeleteRequests(1);
+      return Response.ok().build();
+
+    } catch (IOException e) {
+      servlet.getMetrics().incrementFailedDeleteRequests(1);
+      return processException(e);
+    }
+  }
+
+  /**
+   * Dispatch to NamespaceInstanceResource for getting list of tables.
+   */
+  @Path("tables")
+  public NamespacesInstanceResource getNamespaceInstanceResource(
+      final @PathParam("tables") String namespace) throws IOException {
+    return new NamespacesInstanceResource(this.namespace, true);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java
new file mode 100644
index 0000000..0548fe8
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesResource.java
@@ -0,0 +1,90 @@
+/*
+ *
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.rest;
+
+import java.io.IOException;
+
+import javax.servlet.ServletContext;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.rest.model.NamespacesModel;
+
+/**
+ * Implements REST GET list of all namespaces.
+ * <p>
+ * <tt>/namespaces</tt>
+ * <p>
+ */
+@InterfaceAudience.Private
+public class NamespacesResource extends ResourceBase {
+
+  private static final Log LOG = LogFactory.getLog(NamespacesResource.class);
+
+  /**
+   * Constructor
+   * @throws IOException
+   */
+  public NamespacesResource() throws IOException {
+    super();
+  }
+
+  /**
+   * Build a response for a list of all namespaces request.
+   * @param context servlet context
+   * @param uriInfo (JAX-RS context variable) request URL
+   * @return a response for a version request
+   */
+  @GET
+  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF,
+    MIMETYPE_PROTOBUF_IETF})
+  public Response get(final @Context ServletContext context, final @Context UriInfo uriInfo) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("GET " + uriInfo.getAbsolutePath());
+    }
+    servlet.getMetrics().incrementRequests(1);
+    try {
+      NamespacesModel rowModel = null;
+      rowModel = new NamespacesModel(servlet.getAdmin());
+      servlet.getMetrics().incrementSucessfulGetRequests(1);
+      return Response.ok(rowModel).build();
+    } catch (IOException e) {
+      servlet.getMetrics().incrementFailedGetRequests(1);
+      throw new RuntimeException("Cannot retrieve list of namespaces.");
+    }
+  }
+
+  /**
+   * Dispatch to NamespaceInstanceResource
+   */
+  @Path("{namespace}")
+  public NamespacesInstanceResource getNamespaceInstanceResource(
+      final @PathParam("namespace") String namespace) throws IOException {
+    return new NamespacesInstanceResource(namespace);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
index c425e84..c08bb8b 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java
@@ -103,4 +103,9 @@ public class RootResource extends ResourceBase {
       final @PathParam("table") String table) throws IOException {
     return new TableResource(table);
   }
+
+  @Path("namespaces")
+  public NamespacesResource getNamespaceResource() throws IOException {
+    return new NamespacesResource();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
new file mode 100644
index 0000000..d8528ef
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
@@ -0,0 +1,168 @@
+/*
+ *
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf
+  .generated.NamespacePropertiesMessage.NamespaceProperties;
+
+/**
+ * List a HBase namespace's key/value properties.
+ * <ul>
+ * <li>NamespaceProperties: outer element</li>
+ * <li>properties: sequence property elements</li>
+ * <li>entry</li>
+ * <li>key: property key</li>
+ * <li>value: property value</li>
+ * </ul>
+ */
+@XmlRootElement(name="NamespaceProperties")
+@XmlAccessorType(XmlAccessType.FIELD)
+@InterfaceAudience.Private
+public class NamespacesInstanceModel implements Serializable, ProtobufMessageHandler {
+
+  private static final long serialVersionUID = 1L;
+
+  // JAX-RS automatically converts Map to XMLAnyElement.
+  private Map<String,String> properties = null;
+
+  @XmlTransient
+  private String namespaceName;
+
+  /**
+   * Default constructor. Do not use.
+   */
+  public NamespacesInstanceModel() {}
+
+  /**
+   * Constructor to use if namespace does not exist in HBASE.
+   * @param namespaceName: the namespace name.
+   * @throws IOException
+   */
+  public NamespacesInstanceModel(String namespaceName) throws IOException {
+    this(null, namespaceName);
+  }
+
+  /**
+   * Constructor
+   * @param admin: the administrative API
+   * @param namespaceName: the namespace name.
+   * @throws IOException
+   */
+  public NamespacesInstanceModel(Admin admin, String namespaceName) throws IOException {
+    this.namespaceName = namespaceName;
+    if(admin == null) { return; }
+
+    NamespaceDescriptor nd = admin.getNamespaceDescriptor(namespaceName);
+
+    // For properly formed JSON, if no properties, field has to be null (not just no elements).
+    if(nd.getConfiguration().size() == 0){ return; }
+
+    properties = new HashMap<String,String>();
+    properties.putAll(nd.getConfiguration());
+  }
+
+  /**
+   * Add property to the namespace.
+   * @param key: attribute name
+   * @param value attribute value
+   */
+  public void addProperty(String key, String value) {
+    if(properties == null){
+      properties = new HashMap<String,String>();
+    }
+    properties.put(key, value);
+  }
+
+  /**
+   * @return The map of uncategorized namespace properties.
+   */
+  public Map<String,String> getProperties() {
+    if(properties == null){
+      properties = new HashMap<String,String>();
+    }
+    return properties;
+  }
+
+  public String getNamespaceName(){
+    return namespaceName;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("{NAME => \'");
+    sb.append(namespaceName);
+    sb.append("\'");
+    if(properties != null){
+      for(String key: properties.keySet()){
+        sb.append(", ");
+        sb.append(key);
+        sb.append(" => '");
+        sb.append(properties.get(key));
+        sb.append("\'");
+      }
+    }
+    sb.append("}");
+    return sb.toString();
+  }
+
+  @Override
+  public byte[] createProtobufOutput() {
+    NamespaceProperties.Builder builder = NamespaceProperties.newBuilder();
+    if(properties != null){
+      for(String key: properties.keySet()){
+        NamespaceProperties.Property.Builder property = NamespaceProperties.Property.newBuilder();
+        property.setKey(key);
+        property.setValue(properties.get(key));
+        builder.addProps(property);
+      }
+    }
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message) throws IOException {
+    NamespaceProperties.Builder builder = NamespaceProperties.newBuilder();
+    builder.mergeFrom(message);
+    List<NamespaceProperties.Property> properties = builder.getPropsList();
+    for(NamespaceProperties.Property property: properties){
+      addProperty(property.getKey(), property.getValue());
+    }
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java
new file mode 100644
index 0000000..7b8f3b7
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java
@@ -0,0 +1,116 @@
+/*
+ *
+ * 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.
+ */
+
+package org.apache.hadoop.hbase.rest.model;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
+import org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+
+/**
+ * A list of HBase namespaces.
+ * <ul>
+ * <li>Namespace: namespace name</li>
+ * </ul>
+ */
+@XmlRootElement(name="Namespaces")
+@XmlAccessorType(XmlAccessType.FIELD)
+@InterfaceAudience.Private
+public class NamespacesModel implements Serializable, ProtobufMessageHandler {
+
+  private static final long serialVersionUID = 1L;
+
+  @JsonProperty("Namespace")
+  @XmlElement(name="Namespace")
+  private List<String> namespaces = new ArrayList<String>();
+
+  /**
+   * Default constructor. Do not use.
+   */
+  public NamespacesModel() {}
+
+  /**
+   * Constructor
+   * @param admin: the administrative API
+   * @throws IOException
+   */
+  public NamespacesModel(Admin admin) throws IOException {
+    NamespaceDescriptor[] nds = admin.listNamespaceDescriptors();
+    namespaces = new ArrayList<String>();
+    for (NamespaceDescriptor nd : nds) {
+      namespaces.add(nd.getName());
+    }
+  }
+
+  /**
+   * @return all namespaces
+   */
+  public List<String> getNamespaces() {
+    return namespaces;
+  }
+
+  /**
+   * @param namespaces the namespace name array
+   */
+  public void setNamespaces(List<String> namespaces) {
+    this.namespaces = namespaces;
+  }
+
+  /* (non-Javadoc)
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (String namespace : namespaces) {
+      sb.append(namespace);
+      sb.append("\n");
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public byte[] createProtobufOutput() {
+    Namespaces.Builder builder = Namespaces.newBuilder();
+    builder.addAllNamespace(namespaces);
+    return builder.build().toByteArray();
+  }
+
+  @Override
+  public ProtobufMessageHandler getObjectFromMessage(byte[] message) throws IOException {
+    Namespaces.Builder builder = Namespaces.newBuilder();
+    builder.mergeFrom(message);
+    namespaces = builder.getNamespaceList();
+    return this;
+  }
+}