You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2015/09/10 17:28:08 UTC

[2/5] hbase git commit: Revert "HBASE-14147 Add namespace CRUD functionality to REST."

http://git-wip-us.apache.org/repos/asf/hbase/blob/0bcaa165/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
deleted file mode 100644
index d7649b0..0000000
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
+++ /dev/null
@@ -1,441 +0,0 @@
-/*
- *
- * 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/0bcaa165/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
deleted file mode 100644
index bbd6f32..0000000
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesResource.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- *
- * 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/0bcaa165/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
deleted file mode 100644
index 0d8f544..0000000
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesInstanceModel.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- *
- * 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/0bcaa165/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
deleted file mode 100644
index 12dd004..0000000
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestNamespacesModel.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- *
- * 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));
-  }
-}