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:44 UTC

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
new file mode 100644
index 0000000..bd76be9
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
@@ -0,0 +1,440 @@
+/*
+ *
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.ws.rs.core.MediaType;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.rest.client.Client;
+import org.apache.hadoop.hbase.rest.client.Cluster;
+import org.apache.hadoop.hbase.rest.client.Response;
+import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.model.TableListModel;
+import org.apache.hadoop.hbase.rest.model.TableModel;
+import org.apache.hadoop.hbase.rest.model.TestNamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.provider.JacksonProvider;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import static org.junit.Assert.*;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestNamespacesInstanceResource {
+  private static String NAMESPACE1 = "TestNamespacesInstanceResource1";
+  private static Map<String,String> NAMESPACE1_PROPS = new HashMap<String,String>();
+  private static String NAMESPACE2 = "TestNamespacesInstanceResource2";
+  private static Map<String,String> NAMESPACE2_PROPS = new HashMap<String,String>();
+  private static String NAMESPACE3 = "TestNamespacesInstanceResource3";
+  private static Map<String,String> NAMESPACE3_PROPS = new HashMap<String,String>();
+  private static String NAMESPACE4 = "TestNamespacesInstanceResource4";
+  private static Map<String,String> NAMESPACE4_PROPS = new HashMap<String,String>();
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseRESTTestingUtility REST_TEST_UTIL =
+    new HBaseRESTTestingUtility();
+  private static Client client;
+  private static JAXBContext context;
+  private static Configuration conf;
+  private static TestNamespacesInstanceModel testNamespacesInstanceModel;
+  protected static ObjectMapper jsonMapper;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniCluster();
+    REST_TEST_UTIL.startServletContainer(conf);
+    client = new Client(new Cluster().add("localhost",
+      REST_TEST_UTIL.getServletPort()));
+    testNamespacesInstanceModel = new TestNamespacesInstanceModel();
+    context = JAXBContext.newInstance(NamespacesInstanceModel.class, TableListModel.class);
+    jsonMapper = new JacksonProvider()
+    .locateMapper(NamespacesInstanceModel.class, MediaType.APPLICATION_JSON_TYPE);
+    NAMESPACE1_PROPS.put("key1", "value1");
+    NAMESPACE2_PROPS.put("key2a", "value2a");
+    NAMESPACE2_PROPS.put("key2b", "value2b");
+    NAMESPACE3_PROPS.put("key3", "value3");
+    NAMESPACE4_PROPS.put("key4a", "value4a");
+    NAMESPACE4_PROPS.put("key4b", "value4b");
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    REST_TEST_UTIL.shutdownServletContainer();
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private static byte[] toXML(NamespacesInstanceModel model) throws JAXBException {
+    StringWriter writer = new StringWriter();
+    context.createMarshaller().marshal(model, writer);
+    return Bytes.toBytes(writer.toString());
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> T fromXML(byte[] content)
+      throws JAXBException {
+    return (T) context.createUnmarshaller().unmarshal(new ByteArrayInputStream(content));
+  }
+
+  private NamespaceDescriptor findNamespace(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 nd[i];
+      }
+    }
+    return null;
+  }
+
+  private void checkNamespaceProperties(NamespaceDescriptor nd, Map<String,String> testProps){
+    checkNamespaceProperties(nd.getConfiguration(), testProps);
+  }
+
+  private void checkNamespaceProperties(Map<String,String> namespaceProps, 
+      Map<String,String> testProps){
+    assertTrue(namespaceProps.size() == testProps.size());
+    for(String key: testProps.keySet()){
+      assertEquals(testProps.get(key), namespaceProps.get(key));
+    }
+  }
+
+  private void checkNamespaceTables(List<TableModel> namespaceTables, List<String> testTables){
+    assertEquals(namespaceTables.size(), testTables.size());
+    for(int i = 0 ; i < namespaceTables.size() ; i++){
+      String tableName = ((TableModel) namespaceTables.get(i)).getName();
+      assertTrue(testTables.contains(tableName));
+    }
+  }
+
+  @Test
+  public void testCannotDeleteDefaultAndHbaseNamespaces() throws IOException {
+    String defaultPath = "/namespaces/default";
+    String hbasePath = "/namespaces/hbase";
+    Response response;
+
+    // Check that doesn't exist via non-REST call.
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    assertNotNull(findNamespace(admin, "default"));
+    assertNotNull(findNamespace(admin, "hbase"));
+
+    // Try (but fail) to delete namespaces via REST.
+    response = client.delete(defaultPath);
+    assertEquals(503, response.getCode());
+    response = client.delete(hbasePath);
+    assertEquals(503, response.getCode());
+
+    assertNotNull(findNamespace(admin, "default"));
+    assertNotNull(findNamespace(admin, "hbase"));
+  }
+
+  @Test
+  public void testGetNamespaceTablesAndCannotDeleteNamespace() throws IOException, JAXBException {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    String nsName = "TestNamespacesInstanceResource5";
+    Response response;
+
+    // Create namespace via admin.
+    NamespaceDescriptor.Builder nsBuilder = NamespaceDescriptor.create(nsName);
+    NamespaceDescriptor nsd = nsBuilder.build(); 
+    nsd.setConfiguration("key1", "value1");
+    admin.createNamespace(nsd);
+
+    // Create two tables via admin.
+    HColumnDescriptor colDesc = new HColumnDescriptor("cf1");
+    TableName tn1 = TableName.valueOf(nsName + ":table1");
+    HTableDescriptor table = new HTableDescriptor(tn1);
+    table.addFamily(colDesc);
+    admin.createTable(table);
+    TableName tn2 = TableName.valueOf(nsName + ":table2");
+    table = new HTableDescriptor(tn2);
+    table.addFamily(colDesc);
+    admin.createTable(table);
+
+    Map<String, String> nsProperties = new HashMap<String,String>();
+    nsProperties.put("key1", "value1");
+    List<String> nsTables = Arrays.asList("table1", "table2");
+
+    // Check get namespace properties as XML, JSON and Protobuf.
+    String namespacePath = "/namespaces/" + nsName;
+    response = client.get(namespacePath);
+    assertEquals(200, response.getCode());
+
+    response = client.get(namespacePath, Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    NamespacesInstanceModel model = fromXML(response.getBody());
+    checkNamespaceProperties(model.getProperties(), nsProperties);
+
+    response = client.get(namespacePath, Constants.MIMETYPE_JSON);
+    assertEquals(200, response.getCode());
+    model = jsonMapper.readValue(response.getBody(), NamespacesInstanceModel.class);
+    checkNamespaceProperties(model.getProperties(), nsProperties);
+
+    response = client.get(namespacePath, Constants.MIMETYPE_PROTOBUF);
+    assertEquals(200, response.getCode());
+    model.getObjectFromMessage(response.getBody());
+    checkNamespaceProperties(model.getProperties(), nsProperties);
+
+    // Check get namespace tables as XML, JSON and Protobuf.
+    namespacePath = "/namespaces/" + nsName + "/tables";
+    response = client.get(namespacePath);
+    assertEquals(200, response.getCode());
+
+    response = client.get(namespacePath, Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    TableListModel tablemodel = fromXML(response.getBody());
+    checkNamespaceTables(tablemodel.getTables(), nsTables);
+
+    response = client.get(namespacePath, Constants.MIMETYPE_JSON);
+    assertEquals(200, response.getCode());
+    tablemodel = jsonMapper.readValue(response.getBody(), TableListModel.class);
+    checkNamespaceTables(tablemodel.getTables(), nsTables);
+
+    response = client.get(namespacePath, Constants.MIMETYPE_PROTOBUF);
+    assertEquals(200, response.getCode());
+    tablemodel.setTables(new ArrayList<TableModel>());
+    tablemodel.getObjectFromMessage(response.getBody());
+    checkNamespaceTables(tablemodel.getTables(), nsTables);
+
+    // Check cannot delete namespace via REST because it contains tables.
+    response = client.delete(namespacePath);
+    namespacePath = "/namespaces/" + nsName;
+    assertEquals(503, response.getCode());
+  }
+
+  @Test
+  public void testInvalidNamespacePostsAndPuts() throws IOException, JAXBException {
+    String namespacePath1 = "/namespaces/" + NAMESPACE1;
+    String namespacePath2 = "/namespaces/" + NAMESPACE2;
+    String namespacePath3 = "/namespaces/" + NAMESPACE3;
+    NamespacesInstanceModel model1;
+    NamespacesInstanceModel model2;
+    NamespacesInstanceModel model3;
+    Response response;
+
+    // Check that namespaces don't exist via non-REST call.
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    assertNull(findNamespace(admin, NAMESPACE1));
+    assertNull(findNamespace(admin, NAMESPACE2));
+    assertNull(findNamespace(admin, NAMESPACE3));
+
+    model1 = testNamespacesInstanceModel.buildTestModel(NAMESPACE1, NAMESPACE1_PROPS);
+    testNamespacesInstanceModel.checkModel(model1, NAMESPACE1, NAMESPACE1_PROPS);
+    model2 = testNamespacesInstanceModel.buildTestModel(NAMESPACE2, NAMESPACE2_PROPS);
+    testNamespacesInstanceModel.checkModel(model2, NAMESPACE2, NAMESPACE2_PROPS);
+    model3 = testNamespacesInstanceModel.buildTestModel(NAMESPACE3, NAMESPACE3_PROPS);
+    testNamespacesInstanceModel.checkModel(model3, NAMESPACE3, NAMESPACE3_PROPS);
+
+    // Try REST post and puts with invalid content.
+    response = client.post(namespacePath1, Constants.MIMETYPE_JSON, toXML(model1));
+    assertEquals(500, response.getCode());
+    String jsonString = jsonMapper.writeValueAsString(model2);
+    response = client.put(namespacePath2, Constants.MIMETYPE_XML, Bytes.toBytes(jsonString));
+    assertEquals(400, response.getCode());
+    response = client.post(namespacePath3, Constants.MIMETYPE_PROTOBUF, toXML(model1));
+    assertEquals(500, response.getCode());
+
+    NamespaceDescriptor nd1 = findNamespace(admin, NAMESPACE1);
+    NamespaceDescriptor nd2 = findNamespace(admin, NAMESPACE2);
+    NamespaceDescriptor nd3 = findNamespace(admin, NAMESPACE3);
+    assertNull(nd1);
+    assertNull(nd2);
+    assertNull(nd3);
+  }
+
+  @Test
+  public void testNamespaceCreateAndDeleteXMLAndJSON() throws IOException, JAXBException {
+    String namespacePath1 = "/namespaces/" + NAMESPACE1;
+    String namespacePath2 = "/namespaces/" + NAMESPACE2;
+    NamespacesInstanceModel model1;
+    NamespacesInstanceModel model2;
+    Response response;
+
+    // Check that namespaces don't exist via non-REST call.
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    assertNull(findNamespace(admin, NAMESPACE1));
+    assertNull(findNamespace(admin, NAMESPACE2));
+
+    model1 = testNamespacesInstanceModel.buildTestModel(NAMESPACE1, NAMESPACE1_PROPS);
+    testNamespacesInstanceModel.checkModel(model1, NAMESPACE1, NAMESPACE1_PROPS);
+    model2 = testNamespacesInstanceModel.buildTestModel(NAMESPACE2, NAMESPACE2_PROPS);
+    testNamespacesInstanceModel.checkModel(model2, NAMESPACE2, NAMESPACE2_PROPS);
+
+    // Test cannot PUT (alter) non-existent namespace.
+    response = client.put(namespacePath1, Constants.MIMETYPE_XML, toXML(model1));
+    assertEquals(403, response.getCode());
+    String jsonString = jsonMapper.writeValueAsString(model2);
+    response = client.put(namespacePath2, Constants.MIMETYPE_JSON, Bytes.toBytes(jsonString));
+    assertEquals(403, response.getCode());
+
+    // Test cannot create tables when in read only mode.
+    conf.set("hbase.rest.readonly", "true");
+    response = client.post(namespacePath1, Constants.MIMETYPE_XML, toXML(model1));
+    assertEquals(403, response.getCode());
+    jsonString = jsonMapper.writeValueAsString(model2);
+    response = client.post(namespacePath2, Constants.MIMETYPE_JSON, Bytes.toBytes(jsonString));
+    assertEquals(403, response.getCode());
+    NamespaceDescriptor nd1 = findNamespace(admin, NAMESPACE1);
+    NamespaceDescriptor nd2 = findNamespace(admin, NAMESPACE2);
+    assertNull(nd1);
+    assertNull(nd2);
+    conf.set("hbase.rest.readonly", "false");
+
+    // Create namespace via XML and JSON.
+    response = client.post(namespacePath1, Constants.MIMETYPE_XML, toXML(model1));
+    assertEquals(201, response.getCode());
+    jsonString = jsonMapper.writeValueAsString(model2);
+    response = client.post(namespacePath2, Constants.MIMETYPE_JSON, Bytes.toBytes(jsonString));
+    assertEquals(201, response.getCode());
+
+    // Check that created namespaces correctly.
+    nd1 = findNamespace(admin, NAMESPACE1);
+    nd2 = findNamespace(admin, NAMESPACE2);
+    assertNotNull(nd1);
+    assertNotNull(nd2);
+    checkNamespaceProperties(nd1, NAMESPACE1_PROPS);
+    checkNamespaceProperties(nd1, NAMESPACE1_PROPS);
+
+    // Test cannot delete tables when in read only mode.
+    conf.set("hbase.rest.readonly", "true");
+    response = client.delete(namespacePath1);
+    assertEquals(403, response.getCode());
+    response = client.delete(namespacePath2);
+    assertEquals(403, response.getCode());
+    nd1 = findNamespace(admin, NAMESPACE1);
+    nd2 = findNamespace(admin, NAMESPACE2);
+    assertNotNull(nd1);
+    assertNotNull(nd2);
+    conf.set("hbase.rest.readonly", "false");
+
+    // Delete namespaces via XML and JSON.
+    response = client.delete(namespacePath1);
+    assertEquals(200, response.getCode());
+    response = client.delete(namespacePath2);
+    assertEquals(200, response.getCode());
+    nd1 = findNamespace(admin, NAMESPACE1);
+    nd2 = findNamespace(admin, NAMESPACE2);
+    assertNull(nd1);
+    assertNull(nd2);
+  }
+
+  @Test
+  public void testNamespaceCreateAndDeletePBAndNoBody() throws IOException, JAXBException {
+    String namespacePath3 = "/namespaces/" + NAMESPACE3;
+    String namespacePath4 = "/namespaces/" + NAMESPACE4;
+    NamespacesInstanceModel model3;
+    NamespacesInstanceModel model4;
+    Response response;
+
+    // Check that namespaces don't exist via non-REST call.
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    assertNull(findNamespace(admin, NAMESPACE3));
+    assertNull(findNamespace(admin, NAMESPACE4));
+
+    model3 = testNamespacesInstanceModel.buildTestModel(NAMESPACE3, NAMESPACE3_PROPS);
+    testNamespacesInstanceModel.checkModel(model3, NAMESPACE3, NAMESPACE3_PROPS);
+    model4 = testNamespacesInstanceModel.buildTestModel(NAMESPACE4, NAMESPACE4_PROPS);
+    testNamespacesInstanceModel.checkModel(model4, NAMESPACE4, NAMESPACE4_PROPS);
+
+    // Test cannot PUT (alter) non-existent namespace.
+    response = client.put(namespacePath3, Constants.MIMETYPE_BINARY, new byte[]{});
+    assertEquals(403, response.getCode());
+    response = client.put(namespacePath4, Constants.MIMETYPE_PROTOBUF,
+      model4.createProtobufOutput());
+    assertEquals(403, response.getCode());
+
+    // Test cannot create tables when in read only mode.
+    conf.set("hbase.rest.readonly", "true");
+    response = client.post(namespacePath3, Constants.MIMETYPE_BINARY, new byte[]{});
+    assertEquals(403, response.getCode());
+    response = client.put(namespacePath4, Constants.MIMETYPE_PROTOBUF,
+      model4.createProtobufOutput());
+    assertEquals(403, response.getCode());
+    NamespaceDescriptor nd3 = findNamespace(admin, NAMESPACE3);
+    NamespaceDescriptor nd4 = findNamespace(admin, NAMESPACE4);
+    assertNull(nd3);
+    assertNull(nd4);
+    conf.set("hbase.rest.readonly", "false");
+
+    // Create namespace via no body and protobuf.
+    response = client.post(namespacePath3, Constants.MIMETYPE_BINARY, new byte[]{});
+    assertEquals(201, response.getCode());
+    response = client.post(namespacePath4, Constants.MIMETYPE_PROTOBUF,
+      model4.createProtobufOutput());
+    assertEquals(201, response.getCode());
+
+    // Check that created namespaces correctly.
+    nd3 = findNamespace(admin, NAMESPACE3);
+    nd4 = findNamespace(admin, NAMESPACE4);
+    assertNotNull(nd3);
+    assertNotNull(nd4);
+    checkNamespaceProperties(nd3, new HashMap<String,String>());
+    checkNamespaceProperties(nd4, NAMESPACE4_PROPS);
+
+    // Check cannot post tables that already exist.
+    response = client.post(namespacePath3, Constants.MIMETYPE_BINARY, new byte[]{});
+    assertEquals(403, response.getCode());
+    response = client.post(namespacePath4, Constants.MIMETYPE_PROTOBUF, 
+      model4.createProtobufOutput());
+    assertEquals(403, response.getCode());
+
+    // Check cannot post tables when in read only mode.
+    conf.set("hbase.rest.readonly", "true");
+    response = client.delete(namespacePath3);
+    assertEquals(403, response.getCode());
+    response = client.delete(namespacePath4);
+    assertEquals(403, response.getCode());
+    nd3 = findNamespace(admin, NAMESPACE3);
+    nd4 = findNamespace(admin, NAMESPACE4);
+    assertNotNull(nd3);
+    assertNotNull(nd4);
+    conf.set("hbase.rest.readonly", "false");
+
+    // Delete namespaces via XML and JSON.
+    response = client.delete(namespacePath3);
+    assertEquals(200, response.getCode());
+    response = client.delete(namespacePath4);
+    assertEquals(200, response.getCode());
+    nd3 = findNamespace(admin, NAMESPACE3);
+    nd4 = findNamespace(admin, NAMESPACE4);
+    assertNull(nd3);
+    assertNull(nd4);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
new file mode 100644
index 0000000..a70835f
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
@@ -0,0 +1,203 @@
+/*
+ *
+ * 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.ByteArrayInputStream;
+import java.io.IOException;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.rest.client.Client;
+import org.apache.hadoop.hbase.rest.client.Cluster;
+import org.apache.hadoop.hbase.rest.client.Response;
+import org.apache.hadoop.hbase.rest.model.NamespacesModel;
+import org.apache.hadoop.hbase.rest.model.TestNamespacesModel;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import static org.junit.Assert.*;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestNamespacesResource {
+  private static String NAMESPACE1 = "TestNamespacesInstanceResource1";
+  private static String NAMESPACE2 = "TestNamespacesInstanceResource2";
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseRESTTestingUtility REST_TEST_UTIL =
+    new HBaseRESTTestingUtility();
+  private static Client client;
+  private static JAXBContext context;
+  private static Configuration conf;
+  private static TestNamespacesModel testNamespacesModel;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniCluster();
+    REST_TEST_UTIL.startServletContainer(conf);
+    client = new Client(new Cluster().add("localhost", REST_TEST_UTIL.getServletPort()));
+    testNamespacesModel = new TestNamespacesModel();
+    context = JAXBContext.newInstance(NamespacesModel.class);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    REST_TEST_UTIL.shutdownServletContainer();
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private static NamespacesModel fromXML(byte[] content) throws JAXBException {
+    return (NamespacesModel) context.createUnmarshaller()
+      .unmarshal(new ByteArrayInputStream(content));
+  }
+
+  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;
+  }
+
+  private void createNamespaceViaAdmin(Admin admin, String name) throws IOException {
+    NamespaceDescriptor.Builder builder = NamespaceDescriptor.create(name);
+    NamespaceDescriptor nsd = builder.build();
+    admin.createNamespace(nsd);
+  }
+
+  @Test
+  public void testNamespaceListXMLandJSON() throws IOException, JAXBException {
+    String namespacePath = "/namespaces/";
+    NamespacesModel model;
+    Response response;
+
+    // Check that namespace does not yet exist via non-REST call.
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    assertFalse(doesNamespaceExist(admin, NAMESPACE1));
+    model = testNamespacesModel.buildTestModel();
+    testNamespacesModel.checkModel(model);
+
+    // Check that REST GET finds only default namespaces via XML and JSON responses.
+    response = client.get(namespacePath, Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    model = fromXML(response.getBody());
+    testNamespacesModel.checkModel(model, "hbase", "default");
+    response = client.get(namespacePath, Constants.MIMETYPE_JSON);
+    assertEquals(200, response.getCode());
+    model = testNamespacesModel.fromJSON(Bytes.toString(response.getBody()));
+    testNamespacesModel.checkModel(model, "hbase", "default");
+
+    // Create namespace and check that REST GET finds one additional namespace.
+    createNamespaceViaAdmin(admin, NAMESPACE1);
+    response = client.get(namespacePath, Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    model = fromXML(response.getBody());
+    testNamespacesModel.checkModel(model, NAMESPACE1, "hbase", "default");
+    response = client.get(namespacePath, Constants.MIMETYPE_JSON);
+    assertEquals(200, response.getCode());
+    model = testNamespacesModel.fromJSON(Bytes.toString(response.getBody()));
+    testNamespacesModel.checkModel(model, NAMESPACE1, "hbase", "default");
+
+    // Create another namespace and check that REST GET finds one additional namespace.
+    createNamespaceViaAdmin(admin, NAMESPACE2);
+    response = client.get(namespacePath, Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    model = fromXML(response.getBody());
+    testNamespacesModel.checkModel(model, NAMESPACE1, NAMESPACE2, "hbase", "default");
+    response = client.get(namespacePath, Constants.MIMETYPE_JSON);
+    assertEquals(200, response.getCode());
+    model = testNamespacesModel.fromJSON(Bytes.toString(response.getBody()));
+    testNamespacesModel.checkModel(model, NAMESPACE1, NAMESPACE2, "hbase", "default");
+
+    // Delete namespace and check that REST still finds correct namespaces.
+    admin.deleteNamespace(NAMESPACE1);
+    response = client.get(namespacePath, Constants.MIMETYPE_XML);
+    assertEquals(200, response.getCode());
+    model = fromXML(response.getBody());
+    testNamespacesModel.checkModel(model, NAMESPACE2, "hbase", "default");
+    response = client.get(namespacePath, Constants.MIMETYPE_JSON);
+    assertEquals(200, response.getCode());
+    model = testNamespacesModel.fromJSON(Bytes.toString(response.getBody()));
+    testNamespacesModel.checkModel(model, NAMESPACE2, "hbase", "default");
+
+    admin.deleteNamespace(NAMESPACE2);
+  }
+
+  @Test
+  public void testNamespaceListPBandDefault() throws IOException, JAXBException {
+    String schemaPath = "/namespaces/";
+    NamespacesModel model;
+    Response response;
+
+    // Check that namespace does not yet exist via non-REST call.
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    assertFalse(doesNamespaceExist(admin, NAMESPACE1));
+    model = testNamespacesModel.buildTestModel();
+    testNamespacesModel.checkModel(model);
+
+    // Check that REST GET finds only default namespaces via PB and default Accept header.
+    response = client.get(schemaPath, Constants.MIMETYPE_PROTOBUF);
+    assertEquals(200, response.getCode());
+    model.getObjectFromMessage(response.getBody());
+    testNamespacesModel.checkModel(model, "hbase", "default");
+    response = client.get(schemaPath);
+    assertEquals(200, response.getCode());
+
+    // Create namespace and check that REST GET finds one additional namespace.
+    createNamespaceViaAdmin(admin, NAMESPACE1);
+    response = client.get(schemaPath, Constants.MIMETYPE_PROTOBUF);
+    assertEquals(200, response.getCode());
+    model.getObjectFromMessage(response.getBody());
+    testNamespacesModel.checkModel(model, NAMESPACE1, "hbase", "default");
+    response = client.get(schemaPath);
+    assertEquals(200, response.getCode());
+
+    // Create another namespace and check that REST GET finds one additional namespace.
+    createNamespaceViaAdmin(admin, NAMESPACE2);
+    response = client.get(schemaPath, Constants.MIMETYPE_PROTOBUF);
+    assertEquals(200, response.getCode());
+    model.getObjectFromMessage(response.getBody());
+    testNamespacesModel.checkModel(model, NAMESPACE1, NAMESPACE2, "hbase", "default");
+    response = client.get(schemaPath);
+    assertEquals(200, response.getCode());
+
+    // Delete namespace and check that REST GET still finds correct namespaces.
+    admin.deleteNamespace(NAMESPACE1);
+    response = client.get(schemaPath, Constants.MIMETYPE_PROTOBUF);
+    assertEquals(200, response.getCode());
+    model.getObjectFromMessage(response.getBody());
+    testNamespacesModel.checkModel(model, NAMESPACE2, "hbase", "default");
+    response = client.get(schemaPath);
+    assertEquals(200, response.getCode());
+
+    admin.deleteNamespace(NAMESPACE2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
new file mode 100644
index 0000000..52ed334
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
@@ -0,0 +1,97 @@
+/*
+ *
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestNamespacesInstanceModel extends TestModelBase<NamespacesInstanceModel> {
+
+  public static final Map<String,String> NAMESPACE_PROPERTIES = new HashMap<String, String>();
+  public static final String NAMESPACE_NAME = "namespaceName";
+
+  public TestNamespacesInstanceModel() throws Exception {
+    super(NamespacesInstanceModel.class);
+
+    NAMESPACE_PROPERTIES.put("KEY_1","VALUE_1");
+    NAMESPACE_PROPERTIES.put("KEY_2","VALUE_2");
+    NAMESPACE_PROPERTIES.put("NAME","testNamespace");
+
+    AS_XML =
+      "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>" +
+      "<NamespaceProperties><properties><entry><key>NAME</key><value>testNamespace" +
+      "</value></entry><entry><key>KEY_2</key><value>VALUE_2" +
+      "</value></entry><entry><key>KEY_1</key><value>VALUE_1</value></entry>" +
+      "</properties></NamespaceProperties>";
+
+    AS_PB = "ChUKBE5BTUUSDXRlc3ROYW1lc3BhY2UKEAoFS0VZXzESB1ZBTFVFXzEKEAoFS0VZXzISB1ZBTFVFXzI=";
+
+    AS_JSON = "{\"properties\":{\"NAME\":\"testNamespace\"," +
+      "\"KEY_1\":\"VALUE_1\",\"KEY_2\":\"VALUE_2\"}}";
+  }
+
+  protected NamespacesInstanceModel buildTestModel() {
+    return buildTestModel(NAMESPACE_NAME, NAMESPACE_PROPERTIES);
+  }
+
+  public NamespacesInstanceModel buildTestModel(String namespace, Map<String,String> properties) {
+    NamespacesInstanceModel model = new NamespacesInstanceModel();
+    for(String key: properties.keySet()){
+      model.addProperty(key, properties.get(key));
+    }
+    return model;
+  }
+
+  protected void checkModel(NamespacesInstanceModel model) {
+    checkModel(model, NAMESPACE_NAME, NAMESPACE_PROPERTIES);
+  }
+
+  public void checkModel(NamespacesInstanceModel model, String namespace,
+      Map<String,String> properties) {
+    Map<String,String> modProperties = model.getProperties();
+    assertEquals(properties.size(), modProperties.size());
+    // Namespace name comes from REST URI, not properties.
+    assertNotSame(namespace, model.getNamespaceName());
+    for(String property: properties.keySet()){
+      assertEquals(properties.get(property), modProperties.get(property));
+    }
+  }
+
+  @Test
+  public void testBuildModel() throws Exception {
+    checkModel(buildTestModel());
+  }
+
+  @Test
+  public void testFromXML() throws Exception {
+    checkModel(fromXML(AS_XML));
+  }
+
+  @Test
+  public void testFromPB() throws Exception {
+    checkModel(fromPB(AS_PB));
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java
new file mode 100644
index 0000000..052df43
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java
@@ -0,0 +1,85 @@
+/*
+ *
+ * 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.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+
+@Category(SmallTests.class)
+public class TestNamespacesModel extends TestModelBase<NamespacesModel> {
+
+  public static final String NAMESPACE_NAME_1 = "testNamespace1";
+  public static final String NAMESPACE_NAME_2 = "testNamespace2";
+
+  public TestNamespacesModel() throws Exception {
+    super(NamespacesModel.class);
+
+    AS_XML =
+      "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>" +
+      "<Namespaces><Namespace>testNamespace1</Namespace>" +
+      "<Namespace>testNamespace2</Namespace></Namespaces>";
+
+    AS_PB = "Cg50ZXN0TmFtZXNwYWNlMQoOdGVzdE5hbWVzcGFjZTI=";
+
+    AS_JSON = "{\"Namespace\":[\"testNamespace1\",\"testNamespace2\"]}";
+  }
+
+  protected NamespacesModel buildTestModel() {
+    return buildTestModel(NAMESPACE_NAME_1, NAMESPACE_NAME_2);
+  }
+
+  public NamespacesModel buildTestModel(String... namespaces) {
+    NamespacesModel model = new NamespacesModel();
+    model.setNamespaces(Arrays.asList(namespaces));
+    return model;
+  }
+
+  protected void checkModel(NamespacesModel model) {
+    checkModel(model, NAMESPACE_NAME_1, NAMESPACE_NAME_2);
+  }
+
+  public void checkModel(NamespacesModel model, String... namespaceName) {
+    List<String> namespaces = model.getNamespaces();
+    assertEquals(namespaceName.length, namespaces.size());
+    for(int i = 0; i < namespaceName.length; i++){
+      assertTrue(namespaces.contains(namespaceName[i]));
+    }
+  }
+
+  @Test
+  public void testBuildModel() throws Exception {
+    checkModel(buildTestModel());
+  }
+
+  @Test
+  public void testFromXML() throws Exception {
+    checkModel(fromXML(AS_XML));
+  }
+
+  @Test
+  public void testFromPB() throws Exception {
+    checkModel(fromPB(AS_PB));
+  }
+}