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

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

Repository: hbase
Updated Branches:
  refs/heads/0.98 c4fa84965 -> 75356f2d4
  refs/heads/branch-1 9574c6761 -> 05bd89b9b
  refs/heads/branch-1.2 7e9f7b53a -> cb57ebaa8
  refs/heads/master 8765ffb0c -> 783e20e1a


http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/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..e1990b2
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
@@ -0,0 +1,441 @@
+/*
+ *
+ * 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.HBaseAdmin;
+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(HBaseAdmin 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.
+    HBaseAdmin 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 {
+    HBaseAdmin 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.
+    HBaseAdmin 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.
+    HBaseAdmin 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.
+    HBaseAdmin 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/75356f2d/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..353728f
--- /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.HBaseAdmin;
+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(HBaseAdmin 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(HBaseAdmin 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.
+    HBaseAdmin 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.
+    HBaseAdmin 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/75356f2d/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/75356f2d/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));
+  }
+}


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

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
new file mode 100644
index 0000000..3b5a13a
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
@@ -0,0 +1,1394 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacePropertiesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacePropertiesMessage {
+  private NamespacePropertiesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacePropertiesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> 
+        getPropsList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    int getPropsCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+   */
+  public static final class NamespaceProperties extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacePropertiesOrBuilder {
+    // Use NamespaceProperties.newBuilder() to construct.
+    private NamespaceProperties(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private NamespaceProperties(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final NamespaceProperties defaultInstance;
+    public static NamespaceProperties getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public NamespaceProperties getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private NamespaceProperties(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              props_.add(input.readMessage(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = java.util.Collections.unmodifiableList(props_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<NamespaceProperties> PARSER =
+        new com.google.protobuf.AbstractParser<NamespaceProperties>() {
+      public NamespaceProperties parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new NamespaceProperties(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<NamespaceProperties> getParserForType() {
+      return PARSER;
+    }
+
+    public interface PropertyOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string key = 1;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      boolean hasKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      java.lang.String getKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getKeyBytes();
+
+      // required string value = 2;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      boolean hasValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      java.lang.String getValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getValueBytes();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+     */
+    public static final class Property extends
+        com.google.protobuf.GeneratedMessage
+        implements PropertyOrBuilder {
+      // Use Property.newBuilder() to construct.
+      private Property(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Property(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Property defaultInstance;
+      public static Property getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Property getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Property(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                key_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                value_ = input.readBytes();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Property> PARSER =
+          new com.google.protobuf.AbstractParser<Property>() {
+        public Property parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Property(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Property> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string key = 1;
+      public static final int KEY_FIELD_NUMBER = 1;
+      private java.lang.Object key_;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            key_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required string value = 2;
+      public static final int VALUE_FIELD_NUMBER = 2;
+      private java.lang.Object value_;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public boolean hasValue() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public java.lang.String getValue() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            value_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValueBytes() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          value_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      private void initFields() {
+        key_ = "";
+        value_ = "";
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasKey()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasValue()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBytes(2, getValueBytes());
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(2, getValueBytes());
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          key_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          value_ = "";
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property build() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property buildPartial() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.key_ = key_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.value_ = value_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) {
+            return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property other) {
+          if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance()) return this;
+          if (other.hasKey()) {
+            bitField0_ |= 0x00000001;
+            key_ = other.key_;
+            onChanged();
+          }
+          if (other.hasValue()) {
+            bitField0_ |= 0x00000002;
+            value_ = other.value_;
+            onChanged();
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasKey()) {
+            
+            return false;
+          }
+          if (!hasValue()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string key = 1;
+        private java.lang.Object key_ = "";
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public boolean hasKey() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public java.lang.String getKey() {
+          java.lang.Object ref = key_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            key_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getKeyBytes() {
+          java.lang.Object ref = key_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            key_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKey(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder clearKey() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          key_ = getDefaultInstance().getKey();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKeyBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+
+        // required string value = 2;
+        private java.lang.Object value_ = "";
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public boolean hasValue() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public java.lang.String getValue() {
+          java.lang.Object ref = value_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            value_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public com.google.protobuf.ByteString
+            getValueBytes() {
+          java.lang.Object ref = value_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            value_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValue(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder clearValue() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          value_ = getDefaultInstance().getValue();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValueBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+      }
+
+      static {
+        defaultInstance = new Property(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+    }
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    public static final int PROPS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public int getPropsCount() {
+      return props_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+      return props_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index) {
+      return props_.get(index);
+    }
+
+    private void initFields() {
+      props_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getPropsCount(); i++) {
+        if (!getProps(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < props_.size(); i++) {
+        output.writeMessage(1, props_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < props_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, props_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespacePropertiesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getPropsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties(this);
+        int from_bitField0_ = bitField0_;
+        if (propsBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            props_ = java.util.Collections.unmodifiableList(props_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.props_ = props_;
+        } else {
+          result.props_ = propsBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance()) return this;
+        if (propsBuilder_ == null) {
+          if (!other.props_.isEmpty()) {
+            if (props_.isEmpty()) {
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensurePropsIsMutable();
+              props_.addAll(other.props_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.props_.isEmpty()) {
+            if (propsBuilder_.isEmpty()) {
+              propsBuilder_.dispose();
+              propsBuilder_ = null;
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              propsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getPropsFieldBuilder() : null;
+            } else {
+              propsBuilder_.addAllMessages(other.props_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getPropsCount(); i++) {
+          if (!getProps(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+      private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_ =
+        java.util.Collections.emptyList();
+      private void ensurePropsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>(props_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> propsBuilder_;
+
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+        if (propsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(props_);
+        } else {
+          return propsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public int getPropsCount() {
+        if (propsBuilder_ == null) {
+          return props_.size();
+        } else {
+          return propsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);
+        } else {
+          return propsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.set(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addAllProps(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> values) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          super.addAll(values, props_);
+          onChanged();
+        } else {
+          propsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder clearProps() {
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder removeProps(int index) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.remove(index);
+          onChanged();
+        } else {
+          propsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder getPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+          int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);  } else {
+          return propsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+           getPropsOrBuilderList() {
+        if (propsBuilder_ != null) {
+          return propsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(props_);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder() {
+        return getPropsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder> 
+           getPropsBuilderList() {
+        return getPropsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+          getPropsFieldBuilder() {
+        if (propsBuilder_ == null) {
+          propsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder>(
+                  props_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          props_ = null;
+        }
+        return propsBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+    }
+
+    static {
+      defaultInstance = new NamespaceProperties(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n NamespacePropertiesMessage.proto\022/org." +
+      "apache.hadoop.hbase.rest.protobuf.genera" +
+      "ted\"\233\001\n\023NamespaceProperties\022\\\n\005props\030\001 \003" +
+      "(\0132M.org.apache.hadoop.hbase.rest.protob" +
+      "uf.generated.NamespaceProperties.Propert" +
+      "y\032&\n\010Property\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(" +
+      "\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor,
+              new java.lang.String[] { "Props", });
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor =
+            internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor.getNestedTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor,
+              new java.lang.String[] { "Key", "Value", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
new file mode 100644
index 0000000..20725d2
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
@@ -0,0 +1,547 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacesMessage {
+  private NamespacesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated string namespace = 1;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.util.List<java.lang.String>
+    getNamespaceList();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    int getNamespaceCount();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.lang.String getNamespace(int index);
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNamespaceBytes(int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+   */
+  public static final class Namespaces extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacesOrBuilder {
+    // Use Namespaces.newBuilder() to construct.
+    private Namespaces(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Namespaces(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Namespaces defaultInstance;
+    public static Namespaces getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Namespaces getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Namespaces(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                namespace_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              namespace_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(namespace_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<Namespaces> PARSER =
+        new com.google.protobuf.AbstractParser<Namespaces>() {
+      public Namespaces parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new Namespaces(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<Namespaces> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated string namespace = 1;
+    public static final int NAMESPACE_FIELD_NUMBER = 1;
+    private com.google.protobuf.LazyStringList namespace_;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.util.List<java.lang.String>
+        getNamespaceList() {
+      return namespace_;
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public int getNamespaceCount() {
+      return namespace_.size();
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.lang.String getNamespace(int index) {
+      return namespace_.get(index);
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNamespaceBytes(int index) {
+      return namespace_.getByteString(index);
+    }
+
+    private void initFields() {
+      namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < namespace_.size(); i++) {
+        output.writeBytes(1, namespace_.getByteString(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      {
+        int dataSize = 0;
+        for (int i = 0; i < namespace_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(namespace_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getNamespaceList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.NamespacesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces(this);
+        int from_bitField0_ = bitField0_;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              namespace_);
+          bitField0_ = (bitField0_ & ~0x00000001);
+        }
+        result.namespace_ = namespace_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance()) return this;
+        if (!other.namespace_.isEmpty()) {
+          if (namespace_.isEmpty()) {
+            namespace_ = other.namespace_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensureNamespaceIsMutable();
+            namespace_.addAll(other.namespace_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated string namespace = 1;
+      private com.google.protobuf.LazyStringList namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureNamespaceIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.LazyStringArrayList(namespace_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.util.List<java.lang.String>
+          getNamespaceList() {
+        return java.util.Collections.unmodifiableList(namespace_);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public int getNamespaceCount() {
+        return namespace_.size();
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.lang.String getNamespace(int index) {
+        return namespace_.get(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getNamespaceBytes(int index) {
+        return namespace_.getByteString(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder setNamespace(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespace(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addAllNamespace(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureNamespaceIsMutable();
+        super.addAll(values, namespace_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder clearNamespace() {
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespaceBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+    }
+
+    static {
+      defaultInstance = new Namespaces(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\027NamespacesMessage.proto\022/org.apache.ha" +
+      "doop.hbase.rest.protobuf.generated\"\037\n\nNa" +
+      "mespaces\022\021\n\tnamespace\030\001 \003(\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor,
+              new java.lang.String[] { "Namespace", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
index 3b8b8ca..c9c9639 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
 import org.apache.hadoop.hbase.rest.model.ScannerModel;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
@@ -63,6 +65,8 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
 	  CellModel.class,
     CellSetModel.class,
     ColumnSchemaModel.class,
+    NamespacesModel.class,
+    NamespacesInstanceModel.class,
     RowModel.class,
     ScannerModel.class,
     StorageClusterStatusModel.class,

http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
index ab1b722..53945f3 100644
--- a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
@@ -178,4 +178,32 @@
         <attribute name="currentCompactedKVs" type="int"></attribute>
     </complexType>
 
+    <element name="Namespaces" type="tns:Namespaces"></element>
+
+    <complexType name="Namespaces">
+        <sequence>
+            <element name="Namespace" type="string"
+                maxOccurs="unbounded" minOccurs="0">
+            </element>
+        </sequence>
+    </complexType>
+
+    <complexType name="NamespaceProperties">
+          <sequence>
+               <element name="properties">
+                    <complexType>
+                         <sequence>
+                              <element name="entry" maxOccurs="unbounded" minOccurs="0">
+                                   <complexType>
+                                        <sequence>
+                                             <element type="string" name="key" />
+                                             <element type="string" name="value" />
+                                        </sequence>
+                                   </complexType>
+                              </element>
+                         </sequence>
+                    </complexType>
+               </element>
+          </sequence>
+     </complexType>
 </schema>

http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
new file mode 100644
index 0000000..fbecb71
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
@@ -0,0 +1,26 @@
+//
+// 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.protobuf.generated;
+
+message NamespaceProperties {
+  message Property {
+    required string key = 1;
+    required string value = 2;
+  }
+  repeated Property props = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
new file mode 100644
index 0000000..2c5cbb3
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
@@ -0,0 +1,22 @@
+//
+// 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.protobuf.generated;
+
+message Namespaces {
+	repeated string namespace = 1;
+}
\ No newline at end of file


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

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
new file mode 100644
index 0000000..3b5a13a
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
@@ -0,0 +1,1394 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacePropertiesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacePropertiesMessage {
+  private NamespacePropertiesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacePropertiesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> 
+        getPropsList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    int getPropsCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+   */
+  public static final class NamespaceProperties extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacePropertiesOrBuilder {
+    // Use NamespaceProperties.newBuilder() to construct.
+    private NamespaceProperties(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private NamespaceProperties(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final NamespaceProperties defaultInstance;
+    public static NamespaceProperties getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public NamespaceProperties getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private NamespaceProperties(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              props_.add(input.readMessage(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = java.util.Collections.unmodifiableList(props_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<NamespaceProperties> PARSER =
+        new com.google.protobuf.AbstractParser<NamespaceProperties>() {
+      public NamespaceProperties parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new NamespaceProperties(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<NamespaceProperties> getParserForType() {
+      return PARSER;
+    }
+
+    public interface PropertyOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string key = 1;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      boolean hasKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      java.lang.String getKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getKeyBytes();
+
+      // required string value = 2;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      boolean hasValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      java.lang.String getValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getValueBytes();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+     */
+    public static final class Property extends
+        com.google.protobuf.GeneratedMessage
+        implements PropertyOrBuilder {
+      // Use Property.newBuilder() to construct.
+      private Property(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Property(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Property defaultInstance;
+      public static Property getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Property getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Property(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                key_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                value_ = input.readBytes();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Property> PARSER =
+          new com.google.protobuf.AbstractParser<Property>() {
+        public Property parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Property(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Property> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string key = 1;
+      public static final int KEY_FIELD_NUMBER = 1;
+      private java.lang.Object key_;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            key_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required string value = 2;
+      public static final int VALUE_FIELD_NUMBER = 2;
+      private java.lang.Object value_;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public boolean hasValue() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public java.lang.String getValue() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            value_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValueBytes() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          value_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      private void initFields() {
+        key_ = "";
+        value_ = "";
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasKey()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasValue()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBytes(2, getValueBytes());
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(2, getValueBytes());
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          key_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          value_ = "";
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property build() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property buildPartial() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.key_ = key_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.value_ = value_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) {
+            return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property other) {
+          if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance()) return this;
+          if (other.hasKey()) {
+            bitField0_ |= 0x00000001;
+            key_ = other.key_;
+            onChanged();
+          }
+          if (other.hasValue()) {
+            bitField0_ |= 0x00000002;
+            value_ = other.value_;
+            onChanged();
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasKey()) {
+            
+            return false;
+          }
+          if (!hasValue()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string key = 1;
+        private java.lang.Object key_ = "";
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public boolean hasKey() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public java.lang.String getKey() {
+          java.lang.Object ref = key_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            key_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getKeyBytes() {
+          java.lang.Object ref = key_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            key_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKey(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder clearKey() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          key_ = getDefaultInstance().getKey();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKeyBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+
+        // required string value = 2;
+        private java.lang.Object value_ = "";
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public boolean hasValue() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public java.lang.String getValue() {
+          java.lang.Object ref = value_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            value_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public com.google.protobuf.ByteString
+            getValueBytes() {
+          java.lang.Object ref = value_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            value_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValue(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder clearValue() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          value_ = getDefaultInstance().getValue();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValueBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+      }
+
+      static {
+        defaultInstance = new Property(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+    }
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    public static final int PROPS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public int getPropsCount() {
+      return props_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+      return props_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index) {
+      return props_.get(index);
+    }
+
+    private void initFields() {
+      props_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getPropsCount(); i++) {
+        if (!getProps(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < props_.size(); i++) {
+        output.writeMessage(1, props_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < props_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, props_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespacePropertiesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getPropsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties(this);
+        int from_bitField0_ = bitField0_;
+        if (propsBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            props_ = java.util.Collections.unmodifiableList(props_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.props_ = props_;
+        } else {
+          result.props_ = propsBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance()) return this;
+        if (propsBuilder_ == null) {
+          if (!other.props_.isEmpty()) {
+            if (props_.isEmpty()) {
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensurePropsIsMutable();
+              props_.addAll(other.props_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.props_.isEmpty()) {
+            if (propsBuilder_.isEmpty()) {
+              propsBuilder_.dispose();
+              propsBuilder_ = null;
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              propsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getPropsFieldBuilder() : null;
+            } else {
+              propsBuilder_.addAllMessages(other.props_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getPropsCount(); i++) {
+          if (!getProps(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+      private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_ =
+        java.util.Collections.emptyList();
+      private void ensurePropsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>(props_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> propsBuilder_;
+
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+        if (propsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(props_);
+        } else {
+          return propsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public int getPropsCount() {
+        if (propsBuilder_ == null) {
+          return props_.size();
+        } else {
+          return propsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);
+        } else {
+          return propsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.set(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addAllProps(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> values) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          super.addAll(values, props_);
+          onChanged();
+        } else {
+          propsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder clearProps() {
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder removeProps(int index) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.remove(index);
+          onChanged();
+        } else {
+          propsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder getPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+          int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);  } else {
+          return propsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+           getPropsOrBuilderList() {
+        if (propsBuilder_ != null) {
+          return propsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(props_);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder() {
+        return getPropsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder> 
+           getPropsBuilderList() {
+        return getPropsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+          getPropsFieldBuilder() {
+        if (propsBuilder_ == null) {
+          propsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder>(
+                  props_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          props_ = null;
+        }
+        return propsBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+    }
+
+    static {
+      defaultInstance = new NamespaceProperties(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n NamespacePropertiesMessage.proto\022/org." +
+      "apache.hadoop.hbase.rest.protobuf.genera" +
+      "ted\"\233\001\n\023NamespaceProperties\022\\\n\005props\030\001 \003" +
+      "(\0132M.org.apache.hadoop.hbase.rest.protob" +
+      "uf.generated.NamespaceProperties.Propert" +
+      "y\032&\n\010Property\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(" +
+      "\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor,
+              new java.lang.String[] { "Props", });
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor =
+            internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor.getNestedTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor,
+              new java.lang.String[] { "Key", "Value", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
new file mode 100644
index 0000000..20725d2
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
@@ -0,0 +1,547 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacesMessage {
+  private NamespacesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated string namespace = 1;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.util.List<java.lang.String>
+    getNamespaceList();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    int getNamespaceCount();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.lang.String getNamespace(int index);
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNamespaceBytes(int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+   */
+  public static final class Namespaces extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacesOrBuilder {
+    // Use Namespaces.newBuilder() to construct.
+    private Namespaces(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Namespaces(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Namespaces defaultInstance;
+    public static Namespaces getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Namespaces getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Namespaces(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                namespace_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              namespace_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(namespace_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<Namespaces> PARSER =
+        new com.google.protobuf.AbstractParser<Namespaces>() {
+      public Namespaces parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new Namespaces(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<Namespaces> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated string namespace = 1;
+    public static final int NAMESPACE_FIELD_NUMBER = 1;
+    private com.google.protobuf.LazyStringList namespace_;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.util.List<java.lang.String>
+        getNamespaceList() {
+      return namespace_;
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public int getNamespaceCount() {
+      return namespace_.size();
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.lang.String getNamespace(int index) {
+      return namespace_.get(index);
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNamespaceBytes(int index) {
+      return namespace_.getByteString(index);
+    }
+
+    private void initFields() {
+      namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < namespace_.size(); i++) {
+        output.writeBytes(1, namespace_.getByteString(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      {
+        int dataSize = 0;
+        for (int i = 0; i < namespace_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(namespace_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getNamespaceList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.NamespacesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces(this);
+        int from_bitField0_ = bitField0_;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              namespace_);
+          bitField0_ = (bitField0_ & ~0x00000001);
+        }
+        result.namespace_ = namespace_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance()) return this;
+        if (!other.namespace_.isEmpty()) {
+          if (namespace_.isEmpty()) {
+            namespace_ = other.namespace_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensureNamespaceIsMutable();
+            namespace_.addAll(other.namespace_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated string namespace = 1;
+      private com.google.protobuf.LazyStringList namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureNamespaceIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.LazyStringArrayList(namespace_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.util.List<java.lang.String>
+          getNamespaceList() {
+        return java.util.Collections.unmodifiableList(namespace_);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public int getNamespaceCount() {
+        return namespace_.size();
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.lang.String getNamespace(int index) {
+        return namespace_.get(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getNamespaceBytes(int index) {
+        return namespace_.getByteString(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder setNamespace(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespace(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addAllNamespace(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureNamespaceIsMutable();
+        super.addAll(values, namespace_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder clearNamespace() {
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespaceBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+    }
+
+    static {
+      defaultInstance = new Namespaces(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\027NamespacesMessage.proto\022/org.apache.ha" +
+      "doop.hbase.rest.protobuf.generated\"\037\n\nNa" +
+      "mespaces\022\021\n\tnamespace\030\001 \003(\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor,
+              new java.lang.String[] { "Namespace", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
index 216f830..ba91519 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
 import org.apache.hadoop.hbase.rest.model.ScannerModel;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
@@ -63,6 +65,8 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
     CellModel.class,
     CellSetModel.class,
     ColumnSchemaModel.class,
+    NamespacesModel.class,
+    NamespacesInstanceModel.class,
     RowModel.class,
     ScannerModel.class,
     StorageClusterStatusModel.class,

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
index ab1b722..53945f3 100644
--- a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
@@ -178,4 +178,32 @@
         <attribute name="currentCompactedKVs" type="int"></attribute>
     </complexType>
 
+    <element name="Namespaces" type="tns:Namespaces"></element>
+
+    <complexType name="Namespaces">
+        <sequence>
+            <element name="Namespace" type="string"
+                maxOccurs="unbounded" minOccurs="0">
+            </element>
+        </sequence>
+    </complexType>
+
+    <complexType name="NamespaceProperties">
+          <sequence>
+               <element name="properties">
+                    <complexType>
+                         <sequence>
+                              <element name="entry" maxOccurs="unbounded" minOccurs="0">
+                                   <complexType>
+                                        <sequence>
+                                             <element type="string" name="key" />
+                                             <element type="string" name="value" />
+                                        </sequence>
+                                   </complexType>
+                              </element>
+                         </sequence>
+                    </complexType>
+               </element>
+          </sequence>
+     </complexType>
 </schema>

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
new file mode 100644
index 0000000..fbecb71
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
@@ -0,0 +1,26 @@
+//
+// 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.protobuf.generated;
+
+message NamespaceProperties {
+  message Property {
+    required string key = 1;
+    required string value = 2;
+  }
+  repeated Property props = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
new file mode 100644
index 0000000..2c5cbb3
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
@@ -0,0 +1,22 @@
+//
+// 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.protobuf.generated;
+
+message Namespaces {
+	repeated string namespace = 1;
+}
\ No newline at end of file


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

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/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..d7649b0
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
@@ -0,0 +1,441 @@
+/*
+ *
+ * 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.testclassification.RestTests;
+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({RestTests.class, 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/783e20e1/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..bbd6f32
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
@@ -0,0 +1,204 @@
+/*
+ *
+ * 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.testclassification.RestTests;
+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({RestTests.class, 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);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/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..0d8f544
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
@@ -0,0 +1,98 @@
+/*
+ *
+ * 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.RestTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({RestTests.class, 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/783e20e1/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..12dd004
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java
@@ -0,0 +1,86 @@
+/*
+ *
+ * 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.RestTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+
+@Category({RestTests.class, 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));
+  }
+}


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

Posted by ap...@apache.org.
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));
+  }
+}


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

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/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/05bd89b9/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/05bd89b9/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/05bd89b9/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));
+  }
+}


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

Posted by ap...@apache.org.
HBASE-14147 Add namespace CRUD functionality to REST

Conflicts:
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java

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/cb57ebaa
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/cb57ebaa
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/cb57ebaa

Branch: refs/heads/branch-1.2
Commit: cb57ebaa8a4c1b783df230785e1b6a584d078896
Parents: 7e9f7b5
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:21 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    |  440 ++++++
 .../hbase/rest/TestNamespacesResource.java      |  203 +++
 .../rest/model/TestNamespacesInstanceModel.java |   97 ++
 .../hbase/rest/model/TestNamespacesModel.java   |   85 ++
 16 files changed, 3552 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cb57ebaa/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 5204fb5..f19bc92 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/cb57ebaa/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/cb57ebaa/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/cb57ebaa/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/cb57ebaa/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/cb57ebaa/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;
+  }
+}


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

Posted by ap...@apache.org.
HBASE-14147 Add namespace CRUD functionality to REST

Conflicts:
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java

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/05bd89b9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/05bd89b9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/05bd89b9

Branch: refs/heads/branch-1
Commit: 05bd89b9b781d7c108858d883aae7f08fe1fce32
Parents: 9574c67
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:16 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    |  440 ++++++
 .../hbase/rest/TestNamespacesResource.java      |  203 +++
 .../rest/model/TestNamespacesInstanceModel.java |   97 ++
 .../hbase/rest/model/TestNamespacesModel.java   |   85 ++
 16 files changed, 3552 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 02dad90..75320da 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/05bd89b9/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/05bd89b9/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/05bd89b9/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/05bd89b9/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/05bd89b9/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;
+  }
+}


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

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
new file mode 100644
index 0000000..3b5a13a
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
@@ -0,0 +1,1394 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacePropertiesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacePropertiesMessage {
+  private NamespacePropertiesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacePropertiesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> 
+        getPropsList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    int getPropsCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+   */
+  public static final class NamespaceProperties extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacePropertiesOrBuilder {
+    // Use NamespaceProperties.newBuilder() to construct.
+    private NamespaceProperties(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private NamespaceProperties(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final NamespaceProperties defaultInstance;
+    public static NamespaceProperties getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public NamespaceProperties getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private NamespaceProperties(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              props_.add(input.readMessage(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = java.util.Collections.unmodifiableList(props_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<NamespaceProperties> PARSER =
+        new com.google.protobuf.AbstractParser<NamespaceProperties>() {
+      public NamespaceProperties parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new NamespaceProperties(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<NamespaceProperties> getParserForType() {
+      return PARSER;
+    }
+
+    public interface PropertyOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string key = 1;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      boolean hasKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      java.lang.String getKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getKeyBytes();
+
+      // required string value = 2;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      boolean hasValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      java.lang.String getValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getValueBytes();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+     */
+    public static final class Property extends
+        com.google.protobuf.GeneratedMessage
+        implements PropertyOrBuilder {
+      // Use Property.newBuilder() to construct.
+      private Property(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Property(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Property defaultInstance;
+      public static Property getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Property getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Property(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                key_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                value_ = input.readBytes();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Property> PARSER =
+          new com.google.protobuf.AbstractParser<Property>() {
+        public Property parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Property(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Property> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string key = 1;
+      public static final int KEY_FIELD_NUMBER = 1;
+      private java.lang.Object key_;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            key_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required string value = 2;
+      public static final int VALUE_FIELD_NUMBER = 2;
+      private java.lang.Object value_;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public boolean hasValue() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public java.lang.String getValue() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            value_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValueBytes() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          value_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      private void initFields() {
+        key_ = "";
+        value_ = "";
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasKey()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasValue()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBytes(2, getValueBytes());
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(2, getValueBytes());
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          key_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          value_ = "";
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property build() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property buildPartial() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.key_ = key_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.value_ = value_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) {
+            return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property other) {
+          if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance()) return this;
+          if (other.hasKey()) {
+            bitField0_ |= 0x00000001;
+            key_ = other.key_;
+            onChanged();
+          }
+          if (other.hasValue()) {
+            bitField0_ |= 0x00000002;
+            value_ = other.value_;
+            onChanged();
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasKey()) {
+            
+            return false;
+          }
+          if (!hasValue()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string key = 1;
+        private java.lang.Object key_ = "";
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public boolean hasKey() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public java.lang.String getKey() {
+          java.lang.Object ref = key_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            key_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getKeyBytes() {
+          java.lang.Object ref = key_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            key_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKey(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder clearKey() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          key_ = getDefaultInstance().getKey();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKeyBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+
+        // required string value = 2;
+        private java.lang.Object value_ = "";
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public boolean hasValue() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public java.lang.String getValue() {
+          java.lang.Object ref = value_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            value_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public com.google.protobuf.ByteString
+            getValueBytes() {
+          java.lang.Object ref = value_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            value_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValue(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder clearValue() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          value_ = getDefaultInstance().getValue();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValueBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+      }
+
+      static {
+        defaultInstance = new Property(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+    }
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    public static final int PROPS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public int getPropsCount() {
+      return props_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+      return props_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index) {
+      return props_.get(index);
+    }
+
+    private void initFields() {
+      props_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getPropsCount(); i++) {
+        if (!getProps(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < props_.size(); i++) {
+        output.writeMessage(1, props_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < props_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, props_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespacePropertiesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getPropsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties(this);
+        int from_bitField0_ = bitField0_;
+        if (propsBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            props_ = java.util.Collections.unmodifiableList(props_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.props_ = props_;
+        } else {
+          result.props_ = propsBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance()) return this;
+        if (propsBuilder_ == null) {
+          if (!other.props_.isEmpty()) {
+            if (props_.isEmpty()) {
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensurePropsIsMutable();
+              props_.addAll(other.props_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.props_.isEmpty()) {
+            if (propsBuilder_.isEmpty()) {
+              propsBuilder_.dispose();
+              propsBuilder_ = null;
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              propsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getPropsFieldBuilder() : null;
+            } else {
+              propsBuilder_.addAllMessages(other.props_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getPropsCount(); i++) {
+          if (!getProps(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+      private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_ =
+        java.util.Collections.emptyList();
+      private void ensurePropsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>(props_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> propsBuilder_;
+
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+        if (propsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(props_);
+        } else {
+          return propsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public int getPropsCount() {
+        if (propsBuilder_ == null) {
+          return props_.size();
+        } else {
+          return propsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);
+        } else {
+          return propsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.set(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addAllProps(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> values) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          super.addAll(values, props_);
+          onChanged();
+        } else {
+          propsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder clearProps() {
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder removeProps(int index) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.remove(index);
+          onChanged();
+        } else {
+          propsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder getPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+          int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);  } else {
+          return propsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+           getPropsOrBuilderList() {
+        if (propsBuilder_ != null) {
+          return propsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(props_);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder() {
+        return getPropsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder> 
+           getPropsBuilderList() {
+        return getPropsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+          getPropsFieldBuilder() {
+        if (propsBuilder_ == null) {
+          propsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder>(
+                  props_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          props_ = null;
+        }
+        return propsBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+    }
+
+    static {
+      defaultInstance = new NamespaceProperties(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n NamespacePropertiesMessage.proto\022/org." +
+      "apache.hadoop.hbase.rest.protobuf.genera" +
+      "ted\"\233\001\n\023NamespaceProperties\022\\\n\005props\030\001 \003" +
+      "(\0132M.org.apache.hadoop.hbase.rest.protob" +
+      "uf.generated.NamespaceProperties.Propert" +
+      "y\032&\n\010Property\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(" +
+      "\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor,
+              new java.lang.String[] { "Props", });
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor =
+            internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor.getNestedTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor,
+              new java.lang.String[] { "Key", "Value", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
new file mode 100644
index 0000000..20725d2
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
@@ -0,0 +1,547 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacesMessage {
+  private NamespacesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated string namespace = 1;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.util.List<java.lang.String>
+    getNamespaceList();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    int getNamespaceCount();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.lang.String getNamespace(int index);
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNamespaceBytes(int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+   */
+  public static final class Namespaces extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacesOrBuilder {
+    // Use Namespaces.newBuilder() to construct.
+    private Namespaces(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Namespaces(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Namespaces defaultInstance;
+    public static Namespaces getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Namespaces getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Namespaces(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                namespace_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              namespace_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(namespace_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<Namespaces> PARSER =
+        new com.google.protobuf.AbstractParser<Namespaces>() {
+      public Namespaces parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new Namespaces(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<Namespaces> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated string namespace = 1;
+    public static final int NAMESPACE_FIELD_NUMBER = 1;
+    private com.google.protobuf.LazyStringList namespace_;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.util.List<java.lang.String>
+        getNamespaceList() {
+      return namespace_;
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public int getNamespaceCount() {
+      return namespace_.size();
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.lang.String getNamespace(int index) {
+      return namespace_.get(index);
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNamespaceBytes(int index) {
+      return namespace_.getByteString(index);
+    }
+
+    private void initFields() {
+      namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < namespace_.size(); i++) {
+        output.writeBytes(1, namespace_.getByteString(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      {
+        int dataSize = 0;
+        for (int i = 0; i < namespace_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(namespace_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getNamespaceList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.NamespacesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces(this);
+        int from_bitField0_ = bitField0_;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              namespace_);
+          bitField0_ = (bitField0_ & ~0x00000001);
+        }
+        result.namespace_ = namespace_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance()) return this;
+        if (!other.namespace_.isEmpty()) {
+          if (namespace_.isEmpty()) {
+            namespace_ = other.namespace_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensureNamespaceIsMutable();
+            namespace_.addAll(other.namespace_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated string namespace = 1;
+      private com.google.protobuf.LazyStringList namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureNamespaceIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.LazyStringArrayList(namespace_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.util.List<java.lang.String>
+          getNamespaceList() {
+        return java.util.Collections.unmodifiableList(namespace_);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public int getNamespaceCount() {
+        return namespace_.size();
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.lang.String getNamespace(int index) {
+        return namespace_.get(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getNamespaceBytes(int index) {
+        return namespace_.getByteString(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder setNamespace(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespace(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addAllNamespace(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureNamespaceIsMutable();
+        super.addAll(values, namespace_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder clearNamespace() {
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespaceBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+    }
+
+    static {
+      defaultInstance = new Namespaces(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\027NamespacesMessage.proto\022/org.apache.ha" +
+      "doop.hbase.rest.protobuf.generated\"\037\n\nNa" +
+      "mespaces\022\021\n\tnamespace\030\001 \003(\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor,
+              new java.lang.String[] { "Namespace", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
index 216f830..ba91519 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
 import org.apache.hadoop.hbase.rest.model.ScannerModel;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
@@ -63,6 +65,8 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
     CellModel.class,
     CellSetModel.class,
     ColumnSchemaModel.class,
+    NamespacesModel.class,
+    NamespacesInstanceModel.class,
     RowModel.class,
     ScannerModel.class,
     StorageClusterStatusModel.class,

http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
index ab1b722..53945f3 100644
--- a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
@@ -178,4 +178,32 @@
         <attribute name="currentCompactedKVs" type="int"></attribute>
     </complexType>
 
+    <element name="Namespaces" type="tns:Namespaces"></element>
+
+    <complexType name="Namespaces">
+        <sequence>
+            <element name="Namespace" type="string"
+                maxOccurs="unbounded" minOccurs="0">
+            </element>
+        </sequence>
+    </complexType>
+
+    <complexType name="NamespaceProperties">
+          <sequence>
+               <element name="properties">
+                    <complexType>
+                         <sequence>
+                              <element name="entry" maxOccurs="unbounded" minOccurs="0">
+                                   <complexType>
+                                        <sequence>
+                                             <element type="string" name="key" />
+                                             <element type="string" name="value" />
+                                        </sequence>
+                                   </complexType>
+                              </element>
+                         </sequence>
+                    </complexType>
+               </element>
+          </sequence>
+     </complexType>
 </schema>

http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
new file mode 100644
index 0000000..fbecb71
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
@@ -0,0 +1,26 @@
+//
+// 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.protobuf.generated;
+
+message NamespaceProperties {
+  message Property {
+    required string key = 1;
+    required string value = 2;
+  }
+  repeated Property props = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/05bd89b9/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
new file mode 100644
index 0000000..2c5cbb3
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
@@ -0,0 +1,22 @@
+//
+// 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.protobuf.generated;
+
+message Namespaces {
+	repeated string namespace = 1;
+}
\ No newline at end of file


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

Posted by ap...@apache.org.
HBASE-14147 Add namespace CRUD functionality to REST

Conflicts:
    hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/NamespacesInstanceResource.java
    hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesInstanceModel.java
    hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/NamespacesModel.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
    hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java

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/75356f2d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/75356f2d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/75356f2d

Branch: refs/heads/0.98
Commit: 75356f2d49aeb4d532705ca922b7fe9416e66883
Parents: c4fa849
Author: Matt Warhaftig <mw...@gmail.com>
Authored: Mon Sep 21 18:48:11 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Sep 21 18:48:15 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      |  203 +++
 .../rest/model/TestNamespacesInstanceModel.java |   97 ++
 .../hbase/rest/model/TestNamespacesModel.java   |   85 ++
 16 files changed, 3553 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/75356f2d/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 9192b41..396d4c4 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -365,6 +365,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/75356f2d/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..77eda11
--- /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.HBaseAdmin;
+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();
+    }
+
+    HBaseAdmin 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 HBaseAdmin 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(HBaseAdmin 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{
+      HBaseAdmin 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/75356f2d/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/75356f2d/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/75356f2d/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..ec1d2b2
--- /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.HBaseAdmin;
+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(HBaseAdmin 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/75356f2d/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..042bd48
--- /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.HBaseAdmin;
+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(HBaseAdmin 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;
+  }
+}


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

Posted by ap...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
new file mode 100644
index 0000000..3b5a13a
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacePropertiesMessage.java
@@ -0,0 +1,1394 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacePropertiesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacePropertiesMessage {
+  private NamespacePropertiesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacePropertiesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> 
+        getPropsList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index);
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    int getPropsCount();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList();
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+   */
+  public static final class NamespaceProperties extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacePropertiesOrBuilder {
+    // Use NamespaceProperties.newBuilder() to construct.
+    private NamespaceProperties(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private NamespaceProperties(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final NamespaceProperties defaultInstance;
+    public static NamespaceProperties getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public NamespaceProperties getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private NamespaceProperties(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              props_.add(input.readMessage(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = java.util.Collections.unmodifiableList(props_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<NamespaceProperties> PARSER =
+        new com.google.protobuf.AbstractParser<NamespaceProperties>() {
+      public NamespaceProperties parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new NamespaceProperties(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<NamespaceProperties> getParserForType() {
+      return PARSER;
+    }
+
+    public interface PropertyOrBuilder
+        extends com.google.protobuf.MessageOrBuilder {
+
+      // required string key = 1;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      boolean hasKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      java.lang.String getKey();
+      /**
+       * <code>required string key = 1;</code>
+       */
+      com.google.protobuf.ByteString
+          getKeyBytes();
+
+      // required string value = 2;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      boolean hasValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      java.lang.String getValue();
+      /**
+       * <code>required string value = 2;</code>
+       */
+      com.google.protobuf.ByteString
+          getValueBytes();
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+     */
+    public static final class Property extends
+        com.google.protobuf.GeneratedMessage
+        implements PropertyOrBuilder {
+      // Use Property.newBuilder() to construct.
+      private Property(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+        super(builder);
+        this.unknownFields = builder.getUnknownFields();
+      }
+      private Property(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+      private static final Property defaultInstance;
+      public static Property getDefaultInstance() {
+        return defaultInstance;
+      }
+
+      public Property getDefaultInstanceForType() {
+        return defaultInstance;
+      }
+
+      private final com.google.protobuf.UnknownFieldSet unknownFields;
+      @java.lang.Override
+      public final com.google.protobuf.UnknownFieldSet
+          getUnknownFields() {
+        return this.unknownFields;
+      }
+      private Property(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        initFields();
+        int mutable_bitField0_ = 0;
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+            com.google.protobuf.UnknownFieldSet.newBuilder();
+        try {
+          boolean done = false;
+          while (!done) {
+            int tag = input.readTag();
+            switch (tag) {
+              case 0:
+                done = true;
+                break;
+              default: {
+                if (!parseUnknownField(input, unknownFields,
+                                       extensionRegistry, tag)) {
+                  done = true;
+                }
+                break;
+              }
+              case 10: {
+                bitField0_ |= 0x00000001;
+                key_ = input.readBytes();
+                break;
+              }
+              case 18: {
+                bitField0_ |= 0x00000002;
+                value_ = input.readBytes();
+                break;
+              }
+            }
+          }
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          throw e.setUnfinishedMessage(this);
+        } catch (java.io.IOException e) {
+          throw new com.google.protobuf.InvalidProtocolBufferException(
+              e.getMessage()).setUnfinishedMessage(this);
+        } finally {
+          this.unknownFields = unknownFields.build();
+          makeExtensionsImmutable();
+        }
+      }
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+      }
+
+      public static com.google.protobuf.Parser<Property> PARSER =
+          new com.google.protobuf.AbstractParser<Property>() {
+        public Property parsePartialFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws com.google.protobuf.InvalidProtocolBufferException {
+          return new Property(input, extensionRegistry);
+        }
+      };
+
+      @java.lang.Override
+      public com.google.protobuf.Parser<Property> getParserForType() {
+        return PARSER;
+      }
+
+      private int bitField0_;
+      // required string key = 1;
+      public static final int KEY_FIELD_NUMBER = 1;
+      private java.lang.Object key_;
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public boolean hasKey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public java.lang.String getKey() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            key_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string key = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getKeyBytes() {
+        java.lang.Object ref = key_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          key_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      // required string value = 2;
+      public static final int VALUE_FIELD_NUMBER = 2;
+      private java.lang.Object value_;
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public boolean hasValue() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public java.lang.String getValue() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          return (java.lang.String) ref;
+        } else {
+          com.google.protobuf.ByteString bs = 
+              (com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            value_ = s;
+          }
+          return s;
+        }
+      }
+      /**
+       * <code>required string value = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getValueBytes() {
+        java.lang.Object ref = value_;
+        if (ref instanceof java.lang.String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          value_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+
+      private void initFields() {
+        key_ = "";
+        value_ = "";
+      }
+      private byte memoizedIsInitialized = -1;
+      public final boolean isInitialized() {
+        byte isInitialized = memoizedIsInitialized;
+        if (isInitialized != -1) return isInitialized == 1;
+
+        if (!hasKey()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        if (!hasValue()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+        memoizedIsInitialized = 1;
+        return true;
+      }
+
+      public void writeTo(com.google.protobuf.CodedOutputStream output)
+                          throws java.io.IOException {
+        getSerializedSize();
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          output.writeBytes(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          output.writeBytes(2, getValueBytes());
+        }
+        getUnknownFields().writeTo(output);
+      }
+
+      private int memoizedSerializedSize = -1;
+      public int getSerializedSize() {
+        int size = memoizedSerializedSize;
+        if (size != -1) return size;
+
+        size = 0;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(1, getKeyBytes());
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeBytesSize(2, getValueBytes());
+        }
+        size += getUnknownFields().getSerializedSize();
+        memoizedSerializedSize = size;
+        return size;
+      }
+
+      private static final long serialVersionUID = 0L;
+      @java.lang.Override
+      protected java.lang.Object writeReplace()
+          throws java.io.ObjectStreamException {
+        return super.writeReplace();
+      }
+
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.ByteString data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(byte[] data)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          byte[] data,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return PARSER.parseFrom(data, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(java.io.InputStream input)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseDelimitedFrom(
+          java.io.InputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseDelimitedFrom(input, extensionRegistry);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input);
+      }
+      public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parseFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        return PARSER.parseFrom(input, extensionRegistry);
+      }
+
+      public static Builder newBuilder() { return Builder.create(); }
+      public Builder newBuilderForType() { return newBuilder(); }
+      public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property prototype) {
+        return newBuilder().mergeFrom(prototype);
+      }
+      public Builder toBuilder() { return newBuilder(this); }
+
+      @java.lang.Override
+      protected Builder newBuilderForType(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        Builder builder = new Builder(parent);
+        return builder;
+      }
+      /**
+       * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property}
+       */
+      public static final class Builder extends
+          com.google.protobuf.GeneratedMessage.Builder<Builder>
+         implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder {
+        public static final com.google.protobuf.Descriptors.Descriptor
+            getDescriptor() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+            internalGetFieldAccessorTable() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable
+              .ensureFieldAccessorsInitialized(
+                  org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder.class);
+        }
+
+        // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.newBuilder()
+        private Builder() {
+          maybeForceBuilderInitialization();
+        }
+
+        private Builder(
+            com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+          super(parent);
+          maybeForceBuilderInitialization();
+        }
+        private void maybeForceBuilderInitialization() {
+          if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          }
+        }
+        private static Builder create() {
+          return new Builder();
+        }
+
+        public Builder clear() {
+          super.clear();
+          key_ = "";
+          bitField0_ = (bitField0_ & ~0x00000001);
+          value_ = "";
+          bitField0_ = (bitField0_ & ~0x00000002);
+          return this;
+        }
+
+        public Builder clone() {
+          return create().mergeFrom(buildPartial());
+        }
+
+        public com.google.protobuf.Descriptors.Descriptor
+            getDescriptorForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getDefaultInstanceForType() {
+          return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance();
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property build() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = buildPartial();
+          if (!result.isInitialized()) {
+            throw newUninitializedMessageException(result);
+          }
+          return result;
+        }
+
+        public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property buildPartial() {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property(this);
+          int from_bitField0_ = bitField0_;
+          int to_bitField0_ = 0;
+          if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+            to_bitField0_ |= 0x00000001;
+          }
+          result.key_ = key_;
+          if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+            to_bitField0_ |= 0x00000002;
+          }
+          result.value_ = value_;
+          result.bitField0_ = to_bitField0_;
+          onBuilt();
+          return result;
+        }
+
+        public Builder mergeFrom(com.google.protobuf.Message other) {
+          if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) {
+            return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property)other);
+          } else {
+            super.mergeFrom(other);
+            return this;
+          }
+        }
+
+        public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property other) {
+          if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance()) return this;
+          if (other.hasKey()) {
+            bitField0_ |= 0x00000001;
+            key_ = other.key_;
+            onChanged();
+          }
+          if (other.hasValue()) {
+            bitField0_ |= 0x00000002;
+            value_ = other.value_;
+            onChanged();
+          }
+          this.mergeUnknownFields(other.getUnknownFields());
+          return this;
+        }
+
+        public final boolean isInitialized() {
+          if (!hasKey()) {
+            
+            return false;
+          }
+          if (!hasValue()) {
+            
+            return false;
+          }
+          return true;
+        }
+
+        public Builder mergeFrom(
+            com.google.protobuf.CodedInputStream input,
+            com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+            throws java.io.IOException {
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property parsedMessage = null;
+          try {
+            parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+          } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+            parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property) e.getUnfinishedMessage();
+            throw e;
+          } finally {
+            if (parsedMessage != null) {
+              mergeFrom(parsedMessage);
+            }
+          }
+          return this;
+        }
+        private int bitField0_;
+
+        // required string key = 1;
+        private java.lang.Object key_ = "";
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public boolean hasKey() {
+          return ((bitField0_ & 0x00000001) == 0x00000001);
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public java.lang.String getKey() {
+          java.lang.Object ref = key_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            key_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public com.google.protobuf.ByteString
+            getKeyBytes() {
+          java.lang.Object ref = key_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            key_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKey(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder clearKey() {
+          bitField0_ = (bitField0_ & ~0x00000001);
+          key_ = getDefaultInstance().getKey();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string key = 1;</code>
+         */
+        public Builder setKeyBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+          key_ = value;
+          onChanged();
+          return this;
+        }
+
+        // required string value = 2;
+        private java.lang.Object value_ = "";
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public boolean hasValue() {
+          return ((bitField0_ & 0x00000002) == 0x00000002);
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public java.lang.String getValue() {
+          java.lang.Object ref = value_;
+          if (!(ref instanceof java.lang.String)) {
+            java.lang.String s = ((com.google.protobuf.ByteString) ref)
+                .toStringUtf8();
+            value_ = s;
+            return s;
+          } else {
+            return (java.lang.String) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public com.google.protobuf.ByteString
+            getValueBytes() {
+          java.lang.Object ref = value_;
+          if (ref instanceof String) {
+            com.google.protobuf.ByteString b = 
+                com.google.protobuf.ByteString.copyFromUtf8(
+                    (java.lang.String) ref);
+            value_ = b;
+            return b;
+          } else {
+            return (com.google.protobuf.ByteString) ref;
+          }
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValue(
+            java.lang.String value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder clearValue() {
+          bitField0_ = (bitField0_ & ~0x00000002);
+          value_ = getDefaultInstance().getValue();
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>required string value = 2;</code>
+         */
+        public Builder setValueBytes(
+            com.google.protobuf.ByteString value) {
+          if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+          value_ = value;
+          onChanged();
+          return this;
+        }
+
+        // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+      }
+
+      static {
+        defaultInstance = new Property(true);
+        defaultInstance.initFields();
+      }
+
+      // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property)
+    }
+
+    // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+    public static final int PROPS_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_;
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+        getPropsOrBuilderList() {
+      return props_;
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public int getPropsCount() {
+      return props_.size();
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+      return props_.get(index);
+    }
+    /**
+     * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+     */
+    public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+        int index) {
+      return props_.get(index);
+    }
+
+    private void initFields() {
+      props_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      for (int i = 0; i < getPropsCount(); i++) {
+        if (!getProps(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < props_.size(); i++) {
+        output.writeMessage(1, props_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      for (int i = 0; i < props_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, props_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespacePropertiesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getPropsFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties(this);
+        int from_bitField0_ = bitField0_;
+        if (propsBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            props_ = java.util.Collections.unmodifiableList(props_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.props_ = props_;
+        } else {
+          result.props_ = propsBuilder_.build();
+        }
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.getDefaultInstance()) return this;
+        if (propsBuilder_ == null) {
+          if (!other.props_.isEmpty()) {
+            if (props_.isEmpty()) {
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+            } else {
+              ensurePropsIsMutable();
+              props_.addAll(other.props_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.props_.isEmpty()) {
+            if (propsBuilder_.isEmpty()) {
+              propsBuilder_.dispose();
+              propsBuilder_ = null;
+              props_ = other.props_;
+              bitField0_ = (bitField0_ & ~0x00000001);
+              propsBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getPropsFieldBuilder() : null;
+            } else {
+              propsBuilder_.addAllMessages(other.props_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        for (int i = 0; i < getPropsCount(); i++) {
+          if (!getProps(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;
+      private java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> props_ =
+        java.util.Collections.emptyList();
+      private void ensurePropsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          props_ = new java.util.ArrayList<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property>(props_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> propsBuilder_;
+
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> getPropsList() {
+        if (propsBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(props_);
+        } else {
+          return propsBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public int getPropsCount() {
+        if (propsBuilder_ == null) {
+          return props_.size();
+        } else {
+          return propsBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property getProps(int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);
+        } else {
+          return propsBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.set(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder setProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property value) {
+        if (propsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensurePropsIsMutable();
+          props_.add(index, value);
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addProps(
+          int index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder builderForValue) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          propsBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder addAllProps(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property> values) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          super.addAll(values, props_);
+          onChanged();
+        } else {
+          propsBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder clearProps() {
+        if (propsBuilder_ == null) {
+          props_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          propsBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public Builder removeProps(int index) {
+        if (propsBuilder_ == null) {
+          ensurePropsIsMutable();
+          props_.remove(index);
+          onChanged();
+        } else {
+          propsBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder getPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder getPropsOrBuilder(
+          int index) {
+        if (propsBuilder_ == null) {
+          return props_.get(index);  } else {
+          return propsBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+           getPropsOrBuilderList() {
+        if (propsBuilder_ != null) {
+          return propsBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(props_);
+        }
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder() {
+        return getPropsFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder addPropsBuilder(
+          int index) {
+        return getPropsFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties.Property props = 1;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder> 
+           getPropsBuilderList() {
+        return getPropsFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder> 
+          getPropsFieldBuilder() {
+        if (propsBuilder_ == null) {
+          propsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.Property.Builder, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacePropertiesMessage.NamespaceProperties.PropertyOrBuilder>(
+                  props_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getParentForChildren(),
+                  isClean());
+          props_ = null;
+        }
+        return propsBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+    }
+
+    static {
+      defaultInstance = new NamespaceProperties(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.NamespaceProperties)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n NamespacePropertiesMessage.proto\022/org." +
+      "apache.hadoop.hbase.rest.protobuf.genera" +
+      "ted\"\233\001\n\023NamespaceProperties\022\\\n\005props\030\001 \003" +
+      "(\0132M.org.apache.hadoop.hbase.rest.protob" +
+      "uf.generated.NamespaceProperties.Propert" +
+      "y\032&\n\010Property\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(" +
+      "\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor,
+              new java.lang.String[] { "Props", });
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor =
+            internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_descriptor.getNestedTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_NamespaceProperties_Property_descriptor,
+              new java.lang.String[] { "Key", "Value", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
new file mode 100644
index 0000000..20725d2
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/protobuf/generated/NamespacesMessage.java
@@ -0,0 +1,547 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: NamespacesMessage.proto
+
+package org.apache.hadoop.hbase.rest.protobuf.generated;
+
+public final class NamespacesMessage {
+  private NamespacesMessage() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface NamespacesOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated string namespace = 1;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.util.List<java.lang.String>
+    getNamespaceList();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    int getNamespaceCount();
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    java.lang.String getNamespace(int index);
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getNamespaceBytes(int index);
+  }
+  /**
+   * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+   */
+  public static final class Namespaces extends
+      com.google.protobuf.GeneratedMessage
+      implements NamespacesOrBuilder {
+    // Use Namespaces.newBuilder() to construct.
+    private Namespaces(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Namespaces(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Namespaces defaultInstance;
+    public static Namespaces getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Namespaces getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Namespaces(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                namespace_ = new com.google.protobuf.LazyStringArrayList();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              namespace_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(namespace_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<Namespaces> PARSER =
+        new com.google.protobuf.AbstractParser<Namespaces>() {
+      public Namespaces parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new Namespaces(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<Namespaces> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated string namespace = 1;
+    public static final int NAMESPACE_FIELD_NUMBER = 1;
+    private com.google.protobuf.LazyStringList namespace_;
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.util.List<java.lang.String>
+        getNamespaceList() {
+      return namespace_;
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public int getNamespaceCount() {
+      return namespace_.size();
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public java.lang.String getNamespace(int index) {
+      return namespace_.get(index);
+    }
+    /**
+     * <code>repeated string namespace = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getNamespaceBytes(int index) {
+      return namespace_.getByteString(index);
+    }
+
+    private void initFields() {
+      namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < namespace_.size(); i++) {
+        output.writeBytes(1, namespace_.getByteString(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      {
+        int dataSize = 0;
+        for (int i = 0; i < namespace_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(namespace_.getByteString(i));
+        }
+        size += dataSize;
+        size += 1 * getNamespaceList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.NamespacesOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.class, org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces build() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces buildPartial() {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces result = new org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces(this);
+        int from_bitField0_ = bitField0_;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.UnmodifiableLazyStringList(
+              namespace_);
+          bitField0_ = (bitField0_ & ~0x00000001);
+        }
+        result.namespace_ = namespace_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) {
+          return mergeFrom((org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces other) {
+        if (other == org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces.getDefaultInstance()) return this;
+        if (!other.namespace_.isEmpty()) {
+          if (namespace_.isEmpty()) {
+            namespace_ = other.namespace_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensureNamespaceIsMutable();
+            namespace_.addAll(other.namespace_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.rest.protobuf.generated.NamespacesMessage.Namespaces) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated string namespace = 1;
+      private com.google.protobuf.LazyStringList namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+      private void ensureNamespaceIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          namespace_ = new com.google.protobuf.LazyStringArrayList(namespace_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.util.List<java.lang.String>
+          getNamespaceList() {
+        return java.util.Collections.unmodifiableList(namespace_);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public int getNamespaceCount() {
+        return namespace_.size();
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public java.lang.String getNamespace(int index) {
+        return namespace_.get(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getNamespaceBytes(int index) {
+        return namespace_.getByteString(index);
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder setNamespace(
+          int index, java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespace(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addAllNamespace(
+          java.lang.Iterable<java.lang.String> values) {
+        ensureNamespaceIsMutable();
+        super.addAll(values, namespace_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder clearNamespace() {
+        namespace_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated string namespace = 1;</code>
+       */
+      public Builder addNamespaceBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureNamespaceIsMutable();
+        namespace_.add(value);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+    }
+
+    static {
+      defaultInstance = new Namespaces(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:org.apache.hadoop.hbase.rest.protobuf.generated.Namespaces)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\027NamespacesMessage.proto\022/org.apache.ha" +
+      "doop.hbase.rest.protobuf.generated\"\037\n\nNa" +
+      "mespaces\022\021\n\tnamespace\030\001 \003(\t"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_org_apache_hadoop_hbase_rest_protobuf_generated_Namespaces_descriptor,
+              new java.lang.String[] { "Namespace", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
index 216f830..ba91519 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.rest.model.CellModel;
 import org.apache.hadoop.hbase.rest.model.CellSetModel;
 import org.apache.hadoop.hbase.rest.model.ColumnSchemaModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
+import org.apache.hadoop.hbase.rest.model.NamespacesModel;
 import org.apache.hadoop.hbase.rest.model.RowModel;
 import org.apache.hadoop.hbase.rest.model.ScannerModel;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
@@ -63,6 +65,8 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
     CellModel.class,
     CellSetModel.class,
     ColumnSchemaModel.class,
+    NamespacesModel.class,
+    NamespacesInstanceModel.class,
     RowModel.class,
     ScannerModel.class,
     StorageClusterStatusModel.class,

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
index ab1b722..53945f3 100644
--- a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/XMLSchema.xsd
@@ -178,4 +178,32 @@
         <attribute name="currentCompactedKVs" type="int"></attribute>
     </complexType>
 
+    <element name="Namespaces" type="tns:Namespaces"></element>
+
+    <complexType name="Namespaces">
+        <sequence>
+            <element name="Namespace" type="string"
+                maxOccurs="unbounded" minOccurs="0">
+            </element>
+        </sequence>
+    </complexType>
+
+    <complexType name="NamespaceProperties">
+          <sequence>
+               <element name="properties">
+                    <complexType>
+                         <sequence>
+                              <element name="entry" maxOccurs="unbounded" minOccurs="0">
+                                   <complexType>
+                                        <sequence>
+                                             <element type="string" name="key" />
+                                             <element type="string" name="value" />
+                                        </sequence>
+                                   </complexType>
+                              </element>
+                         </sequence>
+                    </complexType>
+               </element>
+          </sequence>
+     </complexType>
 </schema>

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
new file mode 100644
index 0000000..fbecb71
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacePropertiesMessage.proto
@@ -0,0 +1,26 @@
+//
+// 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.protobuf.generated;
+
+message NamespaceProperties {
+  message Property {
+    required string key = 1;
+    required string value = 2;
+  }
+  repeated Property props = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/783e20e1/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
new file mode 100644
index 0000000..2c5cbb3
--- /dev/null
+++ b/hbase-rest/src/main/resources/org/apache/hadoop/hbase/rest/protobuf/NamespacesMessage.proto
@@ -0,0 +1,22 @@
+//
+// 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.protobuf.generated;
+
+message Namespaces {
+	repeated string namespace = 1;
+}
\ No newline at end of file


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

Posted by ap...@apache.org.
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;
+  }
+}