Hhbase-4503 Purge deprecated HBaseClusterTestCase

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1177843 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-09-30 21:29:32 +00:00
parent ba809c3b29
commit f40479ab63
16 changed files with 565 additions and 1144 deletions

View File

@ -38,6 +38,7 @@ Release 0.92.0 - Unreleased
HBASE-4369 Deprecate HConnection#getZookeeperWatcher in prep for HBASE-1762
HBASE-4247 Add isAborted method to the Abortable interface
(Akash Ashok)
HBASE-4503 Purge deprecated HBaseClusterTestCase
BUG FIXES
HBASE-3280 YouAreDeadException being swallowed in HRS getMaster

View File

@ -1,233 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* 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;
import java.io.File;
import java.io.IOException;
import java.io.PrintWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.util.ReflectionUtils;
/**
* Abstract base class for HBase cluster junit tests. Spins up an hbase
* cluster in setup and tears it down again in tearDown.
* @deprecated Use junit4 and {@link HBaseTestingUtility}
*/
public abstract class HBaseClusterTestCase extends HBaseTestCase {
private static final Log LOG = LogFactory.getLog(HBaseClusterTestCase.class);
public MiniHBaseCluster cluster;
protected MiniDFSCluster dfsCluster;
protected MiniZooKeeperCluster zooKeeperCluster;
protected int regionServers;
protected boolean startDfs;
private boolean openMetaTable = true;
/** default constructor */
public HBaseClusterTestCase() {
this(1);
}
/**
* Start a MiniHBaseCluster with regionServers region servers in-process to
* start with. Also, start a MiniDfsCluster before starting the hbase cluster.
* The configuration used will be edited so that this works correctly.
* @param regionServers number of region servers to start.
*/
public HBaseClusterTestCase(int regionServers) {
this(regionServers, true);
}
/** in-process to
* start with. Optionally, startDfs indicates if a MiniDFSCluster should be
* started. If startDfs is false, the assumption is that an external DFS is
* configured in hbase-site.xml and is already started, or you have started a
* MiniDFSCluster on your own and edited the configuration in memory. (You
* can modify the config used by overriding the preHBaseClusterSetup method.)
* @param regionServers number of region servers to start.
* @param startDfs set to true if MiniDFS should be started
*/
public HBaseClusterTestCase(int regionServers, boolean startDfs) {
super();
this.startDfs = startDfs;
this.regionServers = regionServers;
}
protected void setOpenMetaTable(boolean val) {
openMetaTable = val;
}
/**
* Subclass hook.
*
* Run after dfs is ready but before hbase cluster is started up.
*/
protected void preHBaseClusterSetup() throws Exception {
// continue
}
/**
* Actually start the MiniHBase instance.
*/
protected void hBaseClusterSetup() throws Exception {
File testDir = new File(getUnitTestdir(getName()).toString());
if (testDir.exists()) testDir.delete();
// Note that this is done before we create the MiniHBaseCluster because we
// need to edit the config to add the ZooKeeper servers.
this.zooKeeperCluster = new MiniZooKeeperCluster();
int clientPort = this.zooKeeperCluster.startup(testDir);
conf.set("hbase.zookeeper.property.clientPort", Integer.toString(clientPort));
Configuration c = new Configuration(this.conf);
// start the mini cluster
this.cluster = new MiniHBaseCluster(c, regionServers);
if (openMetaTable) {
// opening the META table ensures that cluster is running
new HTable(c, HConstants.META_TABLE_NAME);
}
}
/**
* Run after hbase cluster is started up.
*/
protected void postHBaseClusterSetup() throws Exception {
// continue
}
@Override
protected void setUp() throws Exception {
try {
if (this.startDfs) {
// This spews a bunch of warnings about missing scheme. TODO: fix.
this.dfsCluster = new MiniDFSCluster(0, this.conf, 2, true, true, true,
null, null, null, null);
// mangle the conf so that the fs parameter points to the minidfs we
// just started up
FileSystem filesystem = dfsCluster.getFileSystem();
conf.set("fs.defaultFS", filesystem.getUri().toString());
Path parentdir = filesystem.getHomeDirectory();
conf.set(HConstants.HBASE_DIR, parentdir.toString());
filesystem.mkdirs(parentdir);
FSUtils.setVersion(filesystem, parentdir);
}
// do the super setup now. if we had done it first, then we would have
// gotten our conf all mangled and a local fs started up.
super.setUp();
// run the pre-cluster setup
preHBaseClusterSetup();
// start the instance
hBaseClusterSetup();
// run post-cluster setup
postHBaseClusterSetup();
} catch (Exception e) {
LOG.error("Exception in setup!", e);
if (cluster != null) {
cluster.shutdown();
}
if (zooKeeperCluster != null) {
zooKeeperCluster.shutdown();
}
if (dfsCluster != null) {
shutdownDfs(dfsCluster);
}
throw e;
}
}
@Override
protected void tearDown() throws Exception {
if (!openMetaTable) {
// open the META table now to ensure cluster is running before shutdown.
new HTable(conf, HConstants.META_TABLE_NAME);
}
super.tearDown();
try {
HConnectionManager.deleteConnection(conf, true);
if (this.cluster != null) {
try {
this.cluster.shutdown();
} catch (Exception e) {
LOG.warn("Closing mini dfs", e);
}
try {
this.zooKeeperCluster.shutdown();
} catch (IOException e) {
LOG.warn("Shutting down ZooKeeper cluster", e);
}
}
if (startDfs) {
shutdownDfs(dfsCluster);
}
} catch (Exception e) {
LOG.error(e);
}
// ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
// "Temporary end-of-test thread dump debugging HADOOP-2040: " + getName());
}
/**
* Use this utility method debugging why cluster won't go down. On a
* period it throws a thread dump. Method ends when all cluster
* regionservers and master threads are no long alive.
*/
public void threadDumpingJoin() {
if (this.cluster.getRegionServerThreads() != null) {
for(Thread t: this.cluster.getRegionServerThreads()) {
threadDumpingJoin(t);
}
}
threadDumpingJoin(this.cluster.getMaster());
}
protected void threadDumpingJoin(final Thread t) {
if (t == null) {
return;
}
long startTime = System.currentTimeMillis();
while (t.isAlive()) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
LOG.info("Continuing...", e);
}
if (System.currentTimeMillis() - startTime > 60000) {
startTime = System.currentTimeMillis();
ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
"Automatic Stack Trace every 60 seconds waiting on " +
t.getName());
}
}
}
}

View File

@ -1,127 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* 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;
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Utility class to build a table of multiple regions.
*/
public class MultiRegionTable extends HBaseClusterTestCase {
protected static final byte [][] KEYS = {
HConstants.EMPTY_BYTE_ARRAY,
Bytes.toBytes("bbb"),
Bytes.toBytes("ccc"),
Bytes.toBytes("ddd"),
Bytes.toBytes("eee"),
Bytes.toBytes("fff"),
Bytes.toBytes("ggg"),
Bytes.toBytes("hhh"),
Bytes.toBytes("iii"),
Bytes.toBytes("jjj"),
Bytes.toBytes("kkk"),
Bytes.toBytes("lll"),
Bytes.toBytes("mmm"),
Bytes.toBytes("nnn"),
Bytes.toBytes("ooo"),
Bytes.toBytes("ppp"),
Bytes.toBytes("qqq"),
Bytes.toBytes("rrr"),
Bytes.toBytes("sss"),
Bytes.toBytes("ttt"),
Bytes.toBytes("uuu"),
Bytes.toBytes("vvv"),
Bytes.toBytes("www"),
Bytes.toBytes("xxx"),
Bytes.toBytes("yyy")
};
protected final byte [] columnFamily;
protected HTableDescriptor desc;
/**
* @param familyName the family to populate.
*/
public MultiRegionTable(final String familyName) {
this(1, familyName);
}
public MultiRegionTable(int nServers, final String familyName) {
super(nServers);
this.columnFamily = Bytes.toBytes(familyName);
// These are needed for the new and improved Map/Reduce framework
System.setProperty("hadoop.log.dir", conf.get("hadoop.log.dir"));
conf.set("mapred.output.dir", conf.get("hadoop.tmp.dir"));
}
/**
* Run after dfs is ready but before hbase cluster is started up.
*/
@Override
protected void preHBaseClusterSetup() throws Exception {
try {
// Create a bunch of regions
FileSystem filesystem = FileSystem.get(conf);
Path rootdir = filesystem.makeQualified(
new Path(conf.get(HConstants.HBASE_DIR)));
filesystem.mkdirs(rootdir);
FSUtils.createTableDescriptor(fs, rootdir, desc);
HRegion[] regions = new HRegion[KEYS.length];
for (int i = 0; i < regions.length; i++) {
int j = (i + 1) % regions.length;
regions[i] = createARegion(KEYS[i], KEYS[j]);
}
// Now create the root and meta regions and insert the data regions
// created above into the meta
createRootAndMetaRegions();
for(int i = 0; i < regions.length; i++) {
HRegion.addRegionToMETA(meta, regions[i]);
}
closeRootAndMeta();
} catch (Exception e) {
shutdownDfs(dfsCluster);
throw e;
}
}
private HRegion createARegion(byte [] startKey, byte [] endKey) throws IOException {
HRegion region = createNewHRegion(desc, startKey, endKey);
addContent(region, this.columnFamily);
closeRegionAndDeleteLog(region);
return region;
}
private void closeRegionAndDeleteLog(HRegion region) throws IOException {
region.close();
region.getLog().closeAndDelete();
}
}

View File

@ -26,32 +26,44 @@ import java.net.URL;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
/**
* Testing, info servers are disabled. This test enables then and checks that
* they serve pages.
*/
public class TestInfoServers extends HBaseClusterTestCase {
public class TestInfoServers {
static final Log LOG = LogFactory.getLog(TestInfoServers.class);
private final static HBaseTestingUtility UTIL = new HBaseTestingUtility();
@Override
protected void preHBaseClusterSetup() {
@BeforeClass
public static void beforeClass() throws Exception {
// The info servers do not run in tests by default.
// Set them to ephemeral ports so they will start
conf.setInt("hbase.master.info.port", 0);
conf.setInt("hbase.regionserver.info.port", 0);
UTIL.getConfiguration().setInt("hbase.master.info.port", 0);
UTIL.getConfiguration().setInt("hbase.regionserver.info.port", 0);
UTIL.startMiniCluster();
}
@AfterClass
public static void afterClass() throws IOException {
UTIL.shutdownMiniCluster();
}
/**
* @throws Exception
*/
@Test
public void testInfoServersRedirect() throws Exception {
// give the cluster time to start up
new HTable(conf, ".META.");
int port = cluster.getMaster().getInfoServer().getPort();
new HTable(UTIL.getConfiguration(), ".META.");
int port = UTIL.getHbaseCluster().getMaster().getInfoServer().getPort();
assertHasExpectedContent(new URL("http://localhost:" + port +
"/index.html"), "master-status");
port = cluster.getRegionServerThreads().get(0).getRegionServer().
port = UTIL.getHbaseCluster().getRegionServerThreads().get(0).getRegionServer().
getInfoServer().getPort();
assertHasExpectedContent(new URL("http://localhost:" + port +
"/index.html"), "rs-status");
@ -64,13 +76,14 @@ public class TestInfoServers extends HBaseClusterTestCase {
* TestMasterStatusServlet, but those are true unit tests
* whereas this uses a cluster.
*/
@Test
public void testInfoServersStatusPages() throws Exception {
// give the cluster time to start up
new HTable(conf, ".META.");
int port = cluster.getMaster().getInfoServer().getPort();
new HTable(UTIL.getConfiguration(), ".META.");
int port = UTIL.getHbaseCluster().getMaster().getInfoServer().getPort();
assertHasExpectedContent(new URL("http://localhost:" + port +
"/master-status"), "META");
port = cluster.getRegionServerThreads().get(0).getRegionServer().
port = UTIL.getHbaseCluster().getRegionServerThreads().get(0).getRegionServer().
getInfoServer().getPort();
assertHasExpectedContent(new URL("http://localhost:" + port +
"/rs-status"), "META");
@ -89,9 +102,7 @@ public class TestInfoServers extends HBaseClusterTestCase {
}
bis.close();
String content = sb.toString();
if (!content.contains(expected)) {
fail("Didn't have expected string '" + expected + "'. Content:\n"
+ content);
}
assertTrue("expected=" + expected + ", content=" + content,
content.contains(expected));
}
}

View File

@ -0,0 +1,321 @@
/**
* 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;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Map;
import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestCase.FlushCache;
import org.apache.hadoop.hbase.HBaseTestCase.HTableIncommon;
import org.apache.hadoop.hbase.HBaseTestCase.Incommon;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Port of old TestScanMultipleVersions, TestTimestamp and TestGetRowVersions
* from old testing framework to {@link HBaseTestingUtility}.
*/
public class TestMultiVersions {
private static final Log LOG = LogFactory.getLog(TestMultiVersions.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private HBaseAdmin admin;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
UTIL.startMiniCluster();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
UTIL.shutdownMiniCluster();
}
@Before
public void before()
throws MasterNotRunningException, ZooKeeperConnectionException {
this.admin = new HBaseAdmin(UTIL.getConfiguration());
}
@After
public void after() throws IOException {
this.admin.close();
}
/**
* Tests user specifiable time stamps putting, getting and scanning. Also
* tests same in presence of deletes. Test cores are written so can be
* run against an HRegion and against an HTable: i.e. both local and remote.
*
* <p>Port of old TestTimestamp test to here so can better utilize the spun
* up cluster running more than a single test per spin up. Keep old tests'
* crazyness.
*/
@Test
public void testTimestamps() throws Exception {
HTableDescriptor desc = new HTableDescriptor("testTimestamps");
desc.addFamily(new HColumnDescriptor(TimestampTestBase.FAMILY_NAME));
this.admin.createTable(desc);
HTable table = new HTable(UTIL.getConfiguration(), desc.getName());
// TODO: Remove these deprecated classes or pull them in here if this is
// only test using them.
Incommon incommon = new HTableIncommon(table);
TimestampTestBase.doTestDelete(incommon, new FlushCache() {
public void flushcache() throws IOException {
UTIL.getHbaseCluster().flushcache();
}
});
// Perhaps drop and readd the table between tests so the former does
// not pollute this latter? Or put into separate tests.
TimestampTestBase.doTestTimestampScanning(incommon, new FlushCache() {
public void flushcache() throws IOException {
UTIL.getMiniHBaseCluster().flushcache();
}
});
}
/**
* Verifies versions across a cluster restart.
* Port of old TestGetRowVersions test to here so can better utilize the spun
* up cluster running more than a single test per spin up. Keep old tests'
* crazyness.
*/
@Test
public void testGetRowVersions() throws Exception {
final String tableName = "testGetRowVersions";
final byte [] contents = Bytes.toBytes("contents");
final byte [] row = Bytes.toBytes("row");
final byte [] value1 = Bytes.toBytes("value1");
final byte [] value2 = Bytes.toBytes("value2");
final long timestamp1 = 100L;
final long timestamp2 = 200L;
final HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(contents));
this.admin.createTable(desc);
Put put = new Put(row, timestamp1, null);
put.add(contents, contents, value1);
HTable table = new HTable(UTIL.getConfiguration(), tableName);
table.put(put);
// Shut down and restart the HBase cluster
UTIL.shutdownMiniHBaseCluster();
LOG.debug("HBase cluster shut down -- restarting");
UTIL.startMiniHBaseCluster(1, 1);
// Make a new connection. Use new Configuration instance because old one
// is tied to an HConnection that has since gone stale.
table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
// Overwrite previous value
put = new Put(row, timestamp2, null);
put.add(contents, contents, value2);
table.put(put);
// Now verify that getRow(row, column, latest) works
Get get = new Get(row);
// Should get one version by default
Result r = table.get(get);
assertNotNull(r);
assertFalse(r.isEmpty());
assertTrue(r.size() == 1);
byte [] value = r.getValue(contents, contents);
assertTrue(value.length != 0);
assertTrue(Bytes.equals(value, value2));
// Now check getRow with multiple versions
get = new Get(row);
get.setMaxVersions();
r = table.get(get);
assertTrue(r.size() == 2);
value = r.getValue(contents, contents);
assertTrue(value.length != 0);
assertTrue(Bytes.equals(value, value2));
NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> map =
r.getMap();
NavigableMap<byte[], NavigableMap<Long, byte[]>> familyMap =
map.get(contents);
NavigableMap<Long, byte[]> versionMap = familyMap.get(contents);
assertTrue(versionMap.size() == 2);
assertTrue(Bytes.equals(value1, versionMap.get(timestamp1)));
assertTrue(Bytes.equals(value2, versionMap.get(timestamp2)));
}
/**
* Port of old TestScanMultipleVersions test here so can better utilize the
* spun up cluster running more than just a single test. Keep old tests
* crazyness.
*
* <p>Tests five cases of scans and timestamps.
* @throws Exception
*/
@Test
public void testScanMultipleVersions() throws Exception {
final byte [] tableName = Bytes.toBytes("testScanMultipleVersions");
final HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
final byte [][] rows = new byte[][] {
Bytes.toBytes("row_0200"),
Bytes.toBytes("row_0800")
};
final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")};
final long [] timestamp = new long[] {100L, 1000L};
this.admin.createTable(desc, splitRows);
HTable table = new HTable(UTIL.getConfiguration(), tableName);
// Assert we got the region layout wanted.
NavigableMap<HRegionInfo, ServerName> locations = table.getRegionLocations();
assertEquals(2, locations.size());
int index = 0;
for (Map.Entry<HRegionInfo, ServerName> e: locations.entrySet()) {
HRegionInfo hri = e.getKey();
if (index == 0) {
assertTrue(Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey()));
assertTrue(Bytes.equals(hri.getEndKey(), splitRows[0]));
} else if (index == 1) {
assertTrue(Bytes.equals(splitRows[0], hri.getStartKey()));
assertTrue(Bytes.equals(hri.getEndKey(), HConstants.EMPTY_END_ROW));
}
index++;
}
// Insert data
for (int i = 0; i < locations.size(); i++) {
for (int j = 0; j < timestamp.length; j++) {
Put put = new Put(rows[i], timestamp[j], null);
put.add(HConstants.CATALOG_FAMILY, null, timestamp[j],
Bytes.toBytes(timestamp[j]));
table.put(put);
}
}
// There are 5 cases we have to test. Each is described below.
for (int i = 0; i < rows.length; i++) {
for (int j = 0; j < timestamp.length; j++) {
Get get = new Get(rows[i]);
get.addFamily(HConstants.CATALOG_FAMILY);
get.setTimeStamp(timestamp[j]);
Result result = table.get(get);
int cellCount = 0;
for(@SuppressWarnings("unused")KeyValue kv : result.list()) {
cellCount++;
}
assertTrue(cellCount == 1);
}
}
// Case 1: scan with LATEST_TIMESTAMP. Should get two rows
int count = 0;
Scan scan = new Scan();
scan.addFamily(HConstants.CATALOG_FAMILY);
ResultScanner s = table.getScanner(scan);
try {
for (Result rr = null; (rr = s.next()) != null;) {
System.out.println(rr.toString());
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 2: Scan with a timestamp greater than most recent timestamp
// (in this case > 1000 and < LATEST_TIMESTAMP. Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeRange(1000L, Long.MAX_VALUE);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = table.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 3: scan with timestamp equal to most recent timestamp
// (in this case == 1000. Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeStamp(1000L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = table.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 4: scan with timestamp greater than first timestamp but less than
// second timestamp (100 < timestamp < 1000). Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeRange(100L, 1000L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = table.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 5: scan with timestamp equal to first timestamp (100)
// Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeStamp(100L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = table.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
}
}

View File

@ -19,82 +19,52 @@
*/
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test whether region rebalancing works. (HBASE-71)
*/
public class TestRegionRebalancing extends HBaseClusterTestCase {
public class TestRegionRebalancing {
final Log LOG = LogFactory.getLog(this.getClass().getName());
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
HTable table;
HTableDescriptor desc;
private static final byte [] FAMILY_NAME = Bytes.toBytes("col");
final byte[] FIVE_HUNDRED_KBYTES;
final byte [] FAMILY_NAME = Bytes.toBytes("col");
/** constructor */
public TestRegionRebalancing() {
super(1);
FIVE_HUNDRED_KBYTES = new byte[500 * 1024];
for (int i = 0; i < 500 * 1024; i++) {
FIVE_HUNDRED_KBYTES[i] = 'x';
}
desc = new HTableDescriptor("test");
desc.addFamily(new HColumnDescriptor(FAMILY_NAME));
@BeforeClass
public static void beforeClass() throws Exception {
UTIL.startMiniCluster(1);
}
/**
* Before the hbase cluster starts up, create some dummy regions.
*/
@Override
public void preHBaseClusterSetup() throws IOException {
// create a 20-region table by writing directly to disk
List<byte []> startKeys = new ArrayList<byte []>();
startKeys.add(null);
for (int i = 10; i < 29; i++) {
startKeys.add(Bytes.toBytes("row_" + i));
}
startKeys.add(null);
LOG.info(startKeys.size() + " start keys generated");
@AfterClass
public static void afterClass() throws IOException {
UTIL.shutdownMiniCluster();
}
List<HRegion> regions = new ArrayList<HRegion>();
for (int i = 0; i < 20; i++) {
regions.add(createAregion(startKeys.get(i), startKeys.get(i+1)));
}
// Now create the root and meta regions and insert the data regions
// created above into the meta
createRootAndMetaRegions();
for (HRegion region : regions) {
HRegion.addRegionToMETA(meta, region);
}
closeRootAndMeta();
// Add new table descriptor file
FSUtils.createTableDescriptor(this.desc, this.conf);
@Before
public void before() {
this.desc = new HTableDescriptor("test");
this.desc.addFamily(new HColumnDescriptor(FAMILY_NAME));
}
/**
@ -103,51 +73,61 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
* @throws IOException
* @throws InterruptedException
*/
public void testRebalancing() throws IOException, InterruptedException {
CatalogTracker ct = new CatalogTracker(conf);
@Test
public void testRebalanceOnRegionServerNumberChange()
throws IOException, InterruptedException {
HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration());
admin.createTable(this.desc, HBaseTestingUtility.KEYS);
this.table = new HTable(UTIL.getConfiguration(), this.desc.getName());
CatalogTracker ct = new CatalogTracker(UTIL.getConfiguration());
ct.start();
Map<HRegionInfo, ServerName> regions = MetaReader.fullScan(ct);
Map<HRegionInfo, ServerName> regions = null;
try {
regions = MetaReader.fullScan(ct);
} finally {
ct.stop();
}
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
LOG.info(e);
}
table = new HTable(conf, "test");
assertEquals("Test table should have 20 regions",
20, table.getStartKeys().length);
assertEquals("Test table should have right number of regions",
HBaseTestingUtility.KEYS.length + 1/*One extra to account for start/end keys*/,
this.table.getStartKeys().length);
// verify that the region assignments are balanced to start out
assertRegionsAreBalanced();
// add a region server - total of 2
LOG.info("Started second server=" +
cluster.startRegionServer().getRegionServer().getServerName());
cluster.getMaster().balance();
UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
UTIL.getHbaseCluster().getMaster().balance();
assertRegionsAreBalanced();
// add a region server - total of 3
LOG.info("Started third server=" +
cluster.startRegionServer().getRegionServer().getServerName());
cluster.getMaster().balance();
UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
UTIL.getHbaseCluster().getMaster().balance();
assertRegionsAreBalanced();
// kill a region server - total of 2
LOG.info("Stopped third server=" + cluster.stopRegionServer(2, false));
cluster.waitOnRegionServer(2);
cluster.getMaster().balance();
LOG.info("Stopped third server=" + UTIL.getHbaseCluster().stopRegionServer(2, false));
UTIL.getHbaseCluster().waitOnRegionServer(2);
UTIL.getHbaseCluster().getMaster().balance();
assertRegionsAreBalanced();
// start two more region servers - total of 4
LOG.info("Readding third server=" +
cluster.startRegionServer().getRegionServer().getServerName());
UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
LOG.info("Added fourth server=" +
cluster.startRegionServer().getRegionServer().getServerName());
cluster.getMaster().balance();
UTIL.getHbaseCluster().startRegionServer().getRegionServer().getServerName());
UTIL.getHbaseCluster().getMaster().balance();
assertRegionsAreBalanced();
for (int i = 0; i < 6; i++){
LOG.info("Adding " + (i + 5) + "th region server");
cluster.startRegionServer();
UTIL.getHbaseCluster().startRegionServer();
}
cluster.getMaster().balance();
UTIL.getHbaseCluster().getMaster().balance();
assertRegionsAreBalanced();
}
@ -169,7 +149,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
// TODO: Fix this test. Old balancer used to run with 'slop'. New
// balancer does not.
boolean success = false;
float slop = (float)conf.getFloat("hbase.regions.slop", 0.1f);
float slop = (float)UTIL.getConfiguration().getFloat("hbase.regions.slop", 0.1f);
if (slop <= 0) slop = 1;
for (int i = 0; i < 5; i++) {
@ -179,7 +159,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
int regionCount = getRegionCount();
List<HRegionServer> servers = getOnlineRegionServers();
double avg = cluster.getMaster().getAverageLoad();
double avg = UTIL.getHbaseCluster().getMaster().getAverageLoad();
int avgLoadPlusSlop = (int)Math.ceil(avg * (1 + slop));
int avgLoadMinusSlop = (int)Math.floor(avg * (1 - slop)) - 1;
LOG.debug("There are " + servers.size() + " servers and " + regionCount
@ -204,7 +184,7 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
Thread.sleep(10000);
} catch (InterruptedException e) {}
cluster.getMaster().balance();
UTIL.getHbaseCluster().getMaster().balance();
continue;
}
@ -218,7 +198,8 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
private List<HRegionServer> getOnlineRegionServers() {
List<HRegionServer> list = new ArrayList<HRegionServer>();
for (JVMClusterUtil.RegionServerThread rst : cluster.getRegionServerThreads()) {
for (JVMClusterUtil.RegionServerThread rst :
UTIL.getHbaseCluster().getRegionServerThreads()) {
if (rst.getRegionServer().isOnline()) {
list.add(rst.getRegionServer());
}
@ -238,20 +219,4 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
} catch (InterruptedException e) {}
}
}
/**
* create a region with the specified start and end key and exactly one row
* inside.
*/
private HRegion createAregion(byte [] startKey, byte [] endKey)
throws IOException {
HRegion region = createNewHRegion(desc, startKey, endKey);
byte [] keyToWrite = startKey == null ? Bytes.toBytes("row_000") : startKey;
Put put = new Put(keyToWrite);
put.add(FAMILY_NAME, null, Bytes.toBytes("test"));
region.put(put);
region.close();
region.getLog().closeAndDelete();
return region;
}
}
}

View File

@ -1,202 +0,0 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* 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;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Regression test for HBASE-613
*/
public class TestScanMultipleVersions extends HBaseClusterTestCase {
private final byte[] TABLE_NAME = Bytes.toBytes("TestScanMultipleVersions");
private final HRegionInfo[] INFOS = new HRegionInfo[2];
private final HRegion[] REGIONS = new HRegion[2];
private final byte[][] ROWS = new byte[][] {
Bytes.toBytes("row_0200"),
Bytes.toBytes("row_0800")
};
private final long[] TIMESTAMPS = new long[] {
100L,
1000L
};
private HTableDescriptor desc = null;
@Override
protected void preHBaseClusterSetup() throws Exception {
testDir = new Path(conf.get(HConstants.HBASE_DIR));
// Create table description in
this.desc = new HTableDescriptor(TABLE_NAME);
this.desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
// Write the table schema to the fs
FSUtils.createTableDescriptor(FileSystem.get(this.conf), this.testDir,
this.desc);
// Region 0 will contain the key range [,row_0500)
INFOS[0] = new HRegionInfo(desc.getName(), HConstants.EMPTY_START_ROW,
Bytes.toBytes("row_0500"));
// Region 1 will contain the key range [row_0500,)
INFOS[1] = new HRegionInfo(desc.getName(), Bytes.toBytes("row_0500"),
HConstants.EMPTY_END_ROW);
// Create root and meta regions
createRootAndMetaRegions();
// Create the regions
for (int i = 0; i < REGIONS.length; i++) {
REGIONS[i] =
HRegion.createHRegion(this.INFOS[i], this.testDir, this.conf,
this.desc);
// Insert data
for (int j = 0; j < TIMESTAMPS.length; j++) {
Put put = new Put(ROWS[i], TIMESTAMPS[j], null);
put.add(HConstants.CATALOG_FAMILY, null, TIMESTAMPS[j],
Bytes.toBytes(TIMESTAMPS[j]));
REGIONS[i].put(put);
}
// Insert the region we created into the meta
HRegion.addRegionToMETA(meta, REGIONS[i]);
// Close region
REGIONS[i].close();
REGIONS[i].getLog().closeAndDelete();
}
// Close root and meta regions
closeRootAndMeta();
}
/**
* @throws Exception
*/
public void testScanMultipleVersions() throws Exception {
// At this point we have created multiple regions and both HDFS and HBase
// are running. There are 5 cases we have to test. Each is described below.
HTable t = new HTable(conf, TABLE_NAME);
for (int i = 0; i < ROWS.length; i++) {
for (int j = 0; j < TIMESTAMPS.length; j++) {
Get get = new Get(ROWS[i]);
get.addFamily(HConstants.CATALOG_FAMILY);
get.setTimeStamp(TIMESTAMPS[j]);
Result result = t.get(get);
int cellCount = 0;
for(@SuppressWarnings("unused")KeyValue kv : result.sorted()) {
cellCount++;
}
assertTrue(cellCount == 1);
}
}
// Case 1: scan with LATEST_TIMESTAMP. Should get two rows
int count = 0;
Scan scan = new Scan();
scan.addFamily(HConstants.CATALOG_FAMILY);
ResultScanner s = t.getScanner(scan);
try {
for (Result rr = null; (rr = s.next()) != null;) {
System.out.println(rr.toString());
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 2: Scan with a timestamp greater than most recent timestamp
// (in this case > 1000 and < LATEST_TIMESTAMP. Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeRange(1000L, Long.MAX_VALUE);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = t.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 3: scan with timestamp equal to most recent timestamp
// (in this case == 1000. Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeStamp(1000L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = t.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 4: scan with timestamp greater than first timestamp but less than
// second timestamp (100 < timestamp < 1000). Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeRange(100L, 1000L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = t.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
// Case 5: scan with timestamp equal to first timestamp (100)
// Should get 2 rows.
count = 0;
scan = new Scan();
scan.setTimeStamp(100L);
scan.addFamily(HConstants.CATALOG_FAMILY);
s = t.getScanner(scan);
try {
while (s.next() != null) {
count += 1;
}
assertEquals("Number of rows should be 2", 2, count);
} finally {
s.close();
}
}
}

View File

@ -36,7 +36,7 @@ public class TimestampTestBase extends HBaseTestCase {
private static final long T1 = 100L;
private static final long T2 = 200L;
private static final byte [] FAMILY_NAME = Bytes.toBytes("colfamily11");
public static final byte [] FAMILY_NAME = Bytes.toBytes("colfamily11");
private static final byte [] QUALIFIER_NAME = Bytes.toBytes("contents");
private static final byte [] ROW = Bytes.toBytes("row");

View File

@ -1,102 +0,0 @@
/**
* Copyright 2009 The Apache Software Foundation
*
* 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.client;
import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Test versions.
* Does shutdown in middle of test to prove versions work across restart.
*/
public class TestGetRowVersions extends HBaseClusterTestCase {
private static final Log LOG = LogFactory.getLog(TestGetRowVersions.class);
private static final String TABLE_NAME = "test";
private static final byte [] CONTENTS = Bytes.toBytes("contents");
private static final byte [] ROW = Bytes.toBytes("row");
private static final byte [] VALUE1 = Bytes.toBytes("value1");
private static final byte [] VALUE2 = Bytes.toBytes("value2");
private static final long TIMESTAMP1 = 100L;
private static final long TIMESTAMP2 = 200L;
@Override
public void setUp() throws Exception {
super.setUp();
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
desc.addFamily(new HColumnDescriptor(CONTENTS));
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
}
/** @throws Exception */
public void testGetRowMultipleVersions() throws Exception {
Put put = new Put(ROW, TIMESTAMP1, null);
put.add(CONTENTS, CONTENTS, VALUE1);
HTable table = new HTable(new Configuration(conf), TABLE_NAME);
table.put(put);
// Shut down and restart the HBase cluster
this.cluster.shutdown();
this.zooKeeperCluster.shutdown();
LOG.debug("HBase cluster shut down -- restarting");
this.hBaseClusterSetup();
// Make a new connection. Use new Configuration instance because old one
// is tied to an HConnection that has since gone statle.
table = new HTable(new Configuration(conf), TABLE_NAME);
// Overwrite previous value
put = new Put(ROW, TIMESTAMP2, null);
put.add(CONTENTS, CONTENTS, VALUE2);
table.put(put);
// Now verify that getRow(row, column, latest) works
Get get = new Get(ROW);
// Should get one version by default
Result r = table.get(get);
assertNotNull(r);
assertFalse(r.isEmpty());
assertTrue(r.size() == 1);
byte [] value = r.getValue(CONTENTS, CONTENTS);
assertTrue(value.length != 0);
assertTrue(Bytes.equals(value, VALUE2));
// Now check getRow with multiple versions
get = new Get(ROW);
get.setMaxVersions();
r = table.get(get);
assertTrue(r.size() == 2);
value = r.getValue(CONTENTS, CONTENTS);
assertTrue(value.length != 0);
assertTrue(Bytes.equals(value, VALUE2));
NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> map =
r.getMap();
NavigableMap<byte[], NavigableMap<Long, byte[]>> familyMap =
map.get(CONTENTS);
NavigableMap<Long, byte[]> versionMap = familyMap.get(CONTENTS);
assertTrue(versionMap.size() == 2);
assertTrue(Bytes.equals(VALUE1, versionMap.get(TIMESTAMP1)));
assertTrue(Bytes.equals(VALUE2, versionMap.get(TIMESTAMP2)));
}
}

View File

@ -1,76 +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.client;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TimestampTestBase;
/**
* Tests user specifiable time stamps putting, getting and scanning. Also
* tests same in presence of deletes. Test cores are written so can be
* run against an HRegion and against an HTable: i.e. both local and remote.
*/
public class TestTimestamp extends HBaseClusterTestCase {
public static String COLUMN_NAME = "colfamily11";
/** constructor */
public TestTimestamp() {
super();
}
/**
* Basic test of timestamps.
* Do the above tests from client side.
* @throws IOException
*/
public void testTimestamps() throws IOException {
HTable t = createTable();
Incommon incommon = new HTableIncommon(t);
TimestampTestBase.doTestDelete(incommon, new FlushCache() {
public void flushcache() throws IOException {
cluster.flushcache();
}
});
// Perhaps drop and readd the table between tests so the former does
// not pollute this latter? Or put into separate tests.
TimestampTestBase.doTestTimestampScanning(incommon, new FlushCache() {
public void flushcache() throws IOException {
cluster.flushcache();
}
});
}
/*
* Create a table named TABLE_NAME.
* @return An instance of an HTable connected to the created table.
* @throws IOException
*/
private HTable createTable() throws IOException {
HTableDescriptor desc = new HTableDescriptor(getName());
desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
return new HTable(conf, getName());
}
}

View File

@ -27,31 +27,38 @@ import java.util.NavigableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MapReduceBase;
import org.apache.hadoop.mapred.MiniMRCluster;
import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.Reporter;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test Map/Reduce job over HBase tables. The map/reduce process we're testing
* on our tables is simple - take every row in the table, reverse the value of
* a particular cell, and write it back to the table.
*/
public class TestTableMapReduce extends MultiRegionTable {
public class TestTableMapReduce {
private static final Log LOG =
LogFactory.getLog(TestTableMapReduce.class.getName());
private static final HBaseTestingUtility UTIL =
new HBaseTestingUtility();
static final String MULTI_REGION_TABLE_NAME = "mrtest";
static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text");
@ -61,12 +68,21 @@ public class TestTableMapReduce extends MultiRegionTable {
OUTPUT_FAMILY
};
/** constructor */
public TestTableMapReduce() {
super(Bytes.toString(INPUT_FAMILY));
desc = new HTableDescriptor(MULTI_REGION_TABLE_NAME);
@BeforeClass
public static void beforeClass() throws Exception {
HTableDescriptor desc = new HTableDescriptor(MULTI_REGION_TABLE_NAME);
desc.addFamily(new HColumnDescriptor(INPUT_FAMILY));
desc.addFamily(new HColumnDescriptor(OUTPUT_FAMILY));
UTIL.startMiniCluster();
HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration());
admin.createTable(desc, HBaseTestingUtility.KEYS);
UTIL.startMiniMapReduceCluster();
}
@AfterClass
public static void afterClass() throws IOException {
UTIL.shutdownMiniMapReduceCluster();
UTIL.shutdownMiniCluster();
}
/**
@ -116,17 +132,16 @@ public class TestTableMapReduce extends MultiRegionTable {
* Test a map/reduce against a multi-region table
* @throws IOException
*/
@Test
public void testMultiRegionTable() throws IOException {
runTestOnTable(new HTable(conf, MULTI_REGION_TABLE_NAME));
runTestOnTable(new HTable(UTIL.getConfiguration(), MULTI_REGION_TABLE_NAME));
}
private void runTestOnTable(HTable table) throws IOException {
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
JobConf jobConf = null;
try {
LOG.info("Before map/reduce startup");
jobConf = new JobConf(conf, TestTableMapReduce.class);
jobConf = new JobConf(UTIL.getConfiguration(), TestTableMapReduce.class);
jobConf.setJobName("process column contents");
jobConf.setNumReduceTasks(1);
TableMapReduceUtil.initTableMapJob(Bytes.toString(table.getTableName()),
@ -142,7 +157,6 @@ public class TestTableMapReduce extends MultiRegionTable {
// verify map-reduce results
verify(Bytes.toString(table.getTableName()));
} finally {
mrCluster.shutdown();
if (jobConf != null) {
FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir")));
}
@ -150,10 +164,10 @@ public class TestTableMapReduce extends MultiRegionTable {
}
private void verify(String tableName) throws IOException {
HTable table = new HTable(conf, tableName);
HTable table = new HTable(UTIL.getConfiguration(), tableName);
boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt("hbase.client.retries.number", 5);
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt("hbase.client.retries.number", 5);
for (int i = 0; i < numRetries; i++) {
try {
LOG.info("Verification attempt #" + i);
@ -171,7 +185,7 @@ public class TestTableMapReduce extends MultiRegionTable {
// continue
}
}
assertTrue(verified);
org.junit.Assert.assertTrue(verified);
}
/**
@ -234,7 +248,7 @@ public class TestTableMapReduce extends MultiRegionTable {
r.getRow() + ", first value=" + first + ", second value=" +
second);
}
fail();
org.junit.Assert.fail();
}
}
} finally {

View File

@ -29,11 +29,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MultiRegionTable;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -41,27 +42,43 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.MiniMRCluster;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.fail;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
/**
* Test Map/Reduce job over HBase tables. The map/reduce process we're testing
* on our tables is simple - take every row in the table, reverse the value of
* a particular cell, and write it back to the table.
*/
public class TestTableMapReduce extends MultiRegionTable {
public class TestTableMapReduce {
private static final Log LOG = LogFactory.getLog(TestTableMapReduce.class);
private static final HBaseTestingUtility UTIL =
new HBaseTestingUtility();
static final String MULTI_REGION_TABLE_NAME = "mrtest";
static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
static final byte[] OUTPUT_FAMILY = Bytes.toBytes("text");
/** constructor */
public TestTableMapReduce() {
super(Bytes.toString(INPUT_FAMILY));
desc = new HTableDescriptor(MULTI_REGION_TABLE_NAME);
@BeforeClass
public static void beforeClass() throws Exception {
HTableDescriptor desc = new HTableDescriptor(MULTI_REGION_TABLE_NAME);
desc.addFamily(new HColumnDescriptor(INPUT_FAMILY));
desc.addFamily(new HColumnDescriptor(OUTPUT_FAMILY));
UTIL.startMiniCluster();
HBaseAdmin admin = new HBaseAdmin(UTIL.getConfiguration());
admin.createTable(desc, HBaseTestingUtility.KEYS);
UTIL.startMiniMapReduceCluster();
}
@AfterClass
public static void afterClass() throws IOException {
UTIL.shutdownMiniMapReduceCluster();
UTIL.shutdownMiniCluster();
}
/**
@ -109,14 +126,15 @@ public class TestTableMapReduce extends MultiRegionTable {
* @throws ClassNotFoundException
* @throws InterruptedException
*/
@Test
public void testMultiRegionTable()
throws IOException, InterruptedException, ClassNotFoundException {
runTestOnTable(new HTable(new Configuration(conf), MULTI_REGION_TABLE_NAME));
runTestOnTable(new HTable(new Configuration(UTIL.getConfiguration()),
MULTI_REGION_TABLE_NAME));
}
private void runTestOnTable(HTable table)
throws IOException, InterruptedException, ClassNotFoundException {
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
Job job = null;
try {
LOG.info("Before map/reduce startup");
@ -140,7 +158,6 @@ public class TestTableMapReduce extends MultiRegionTable {
verify(Bytes.toString(table.getTableName()));
} finally {
table.close();
mrCluster.shutdown();
if (job != null) {
FileUtil.fullyDelete(
new File(job.getConfiguration().get("hadoop.tmp.dir")));
@ -149,10 +166,10 @@ public class TestTableMapReduce extends MultiRegionTable {
}
private void verify(String tableName) throws IOException {
HTable table = new HTable(new Configuration(conf), tableName);
HTable table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt("hbase.client.retries.number", 5);
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt("hbase.client.retries.number", 5);
for (int i = 0; i < numRetries; i++) {
try {
LOG.info("Verification attempt #" + i);

View File

@ -32,11 +32,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
@ -47,13 +49,19 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.MapWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.MiniMRCluster;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
public class TestTimeRangeMapRed extends HBaseClusterTestCase {
public class TestTimeRangeMapRed {
private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class);
private static final HBaseTestingUtility UTIL =
new HBaseTestingUtility();
private HBaseAdmin admin;
private static final byte [] KEY = Bytes.toBytes("row1");
private static final NavigableMap<Long, Boolean> TIMESTAMP =
@ -74,26 +82,28 @@ public class TestTimeRangeMapRed extends HBaseClusterTestCase {
static final byte[] FAMILY_NAME = Bytes.toBytes("text");
static final byte[] COLUMN_NAME = Bytes.toBytes("input");
protected HTableDescriptor desc;
protected HTable table;
public TestTimeRangeMapRed() {
super();
System.setProperty("hadoop.log.dir", conf.get("hadoop.log.dir"));
conf.set("mapred.output.dir", conf.get("hadoop.tmp.dir"));
this.setOpenMetaTable(true);
@BeforeClass
public static void beforeClass() throws Exception {
System.setProperty("hadoop.log.dir",
UTIL.getConfiguration().get("hadoop.log.dir"));
UTIL.getConfiguration().set("mapred.output.dir",
UTIL.getConfiguration().get("hadoop.tmp.dir"));
UTIL.startMiniCluster();
}
@Override
public void setUp() throws Exception {
super.setUp();
desc = new HTableDescriptor(TABLE_NAME);
HColumnDescriptor col = new HColumnDescriptor(FAMILY_NAME);
col.setMaxVersions(Integer.MAX_VALUE);
desc.addFamily(col);
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
table = new HTable(conf, desc.getName());
@AfterClass
public static void afterClass() throws Exception {
UTIL.shutdownMiniCluster();
}
@Before
public void before() throws MasterNotRunningException, ZooKeeperConnectionException {
this.admin = new HBaseAdmin(UTIL.getConfiguration());
}
@After
public void after() throws IOException {
this.admin.close();
}
private static class ProcessTimeRangeMapper
@ -108,7 +118,7 @@ public class TestTimeRangeMapRed extends HBaseClusterTestCase {
Context context)
throws IOException {
List<Long> tsList = new ArrayList<Long>();
for (KeyValue kv : result.sorted()) {
for (KeyValue kv : result.list()) {
tsList.add(kv.getTimestamp());
}
@ -134,17 +144,23 @@ public class TestTimeRangeMapRed extends HBaseClusterTestCase {
e.printStackTrace();
}
}
}
@Test
public void testTimeRangeMapRed()
throws IOException, InterruptedException, ClassNotFoundException {
prepareTest();
final HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
final HColumnDescriptor col = new HColumnDescriptor(FAMILY_NAME);
col.setMaxVersions(Integer.MAX_VALUE);
desc.addFamily(col);
admin.createTable(desc);
HTable table = new HTable(UTIL.getConfiguration(), desc.getName());
prepareTest(table);
runTestOnTable();
verify();
verify(table);
}
private void prepareTest() throws IOException {
private void prepareTest(final HTable table) throws IOException {
for (Map.Entry<Long, Boolean> entry : TIMESTAMP.entrySet()) {
Put put = new Put(KEY);
put.add(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false));
@ -155,10 +171,10 @@ public class TestTimeRangeMapRed extends HBaseClusterTestCase {
private void runTestOnTable()
throws IOException, InterruptedException, ClassNotFoundException {
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
UTIL.startMiniMapReduceCluster(1);
Job job = null;
try {
job = new Job(conf, "test123");
job = new Job(UTIL.getConfiguration(), "test123");
job.setOutputFormatClass(NullOutputFormat.class);
job.setNumReduceTasks(0);
Scan scan = new Scan();
@ -172,7 +188,7 @@ public class TestTimeRangeMapRed extends HBaseClusterTestCase {
// TODO Auto-generated catch block
e.printStackTrace();
} finally {
mrCluster.shutdown();
UTIL.shutdownMiniMapReduceCluster();
if (job != null) {
FileUtil.fullyDelete(
new File(job.getConfiguration().get("hadoop.tmp.dir")));
@ -180,20 +196,20 @@ public class TestTimeRangeMapRed extends HBaseClusterTestCase {
}
}
private void verify() throws IOException {
private void verify(final HTable table) throws IOException {
Scan scan = new Scan();
scan.addColumn(FAMILY_NAME, COLUMN_NAME);
scan.setMaxVersions(1);
ResultScanner scanner = table.getScanner(scan);
for (Result r: scanner) {
for (KeyValue kv : r.sorted()) {
for (KeyValue kv : r.list()) {
log.debug(Bytes.toString(r.getRow()) + "\t" + Bytes.toString(kv.getFamily())
+ "\t" + Bytes.toString(kv.getQualifier())
+ "\t" + kv.getTimestamp() + "\t" + Bytes.toBoolean(kv.getValue()));
assertEquals(TIMESTAMP.get(kv.getTimestamp()), (Boolean)Bytes.toBoolean(kv.getValue()));
org.junit.Assert.assertEquals(TIMESTAMP.get(kv.getTimestamp()),
(Boolean)Bytes.toBoolean(kv.getValue()));
}
}
scanner.close();
}
}
}

View File

@ -20,10 +20,19 @@
package org.apache.hadoop.hbase.master;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.*;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Random;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -41,7 +50,6 @@ import org.junit.Test;
*/
public class TestDefaultLoadBalancer {
private static final Log LOG = LogFactory.getLog(TestDefaultLoadBalancer.class);
private static final Random RANDOM = new Random(System.currentTimeMillis());
private static LoadBalancer loadBalancer;
@ -363,10 +371,6 @@ public class TestDefaultLoadBalancer {
return list;
}
private String printMock(Map<ServerName, List<HRegionInfo>> servers) {
return printMock(convertToList(servers));
}
private String printMock(List<ServerAndLoad> balancedCluster) {
SortedSet<ServerAndLoad> sorted =
new TreeSet<ServerAndLoad>(balancedCluster);

View File

@ -1,211 +0,0 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* 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.regionserver;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
/**
* Tests region server failover when a region server exits both cleanly and
* when it aborts.
*/
public class DisabledTestRegionServerExit extends HBaseClusterTestCase {
final Log LOG = LogFactory.getLog(this.getClass().getName());
HTable table;
/** constructor */
public DisabledTestRegionServerExit() {
super(2);
conf.setInt("ipc.client.connect.max.retries", 5); // reduce ipc retries
conf.setInt("ipc.client.timeout", 10000); // and ipc timeout
conf.setInt("hbase.client.pause", 10000); // increase client timeout
conf.setInt("hbase.client.retries.number", 10); // increase HBase retries
}
/**
* Test abort of region server.
* @throws IOException
*/
public void testAbort() throws IOException {
// When the META table can be opened, the region servers are running
new HTable(conf, HConstants.META_TABLE_NAME);
// Create table and add a row.
final String tableName = getName();
byte [] row = createTableAndAddRow(tableName);
// Start up a new region server to take over serving of root and meta
// after we shut down the current meta/root host.
this.cluster.startRegionServer();
// Now abort the meta region server and wait for it to go down and come back
stopOrAbortMetaRegionServer(true);
// Verify that everything is back up.
LOG.info("Starting up the verification thread for " + getName());
Thread t = startVerificationThread(tableName, row);
t.start();
threadDumpingJoin(t);
}
/**
* Test abort of region server.
* Test is flakey up on hudson. Needs work.
* @throws IOException
*/
public void testCleanExit() throws IOException {
// When the META table can be opened, the region servers are running
new HTable(this.conf, HConstants.META_TABLE_NAME);
// Create table and add a row.
final String tableName = getName();
byte [] row = createTableAndAddRow(tableName);
// Start up a new region server to take over serving of root and meta
// after we shut down the current meta/root host.
this.cluster.startRegionServer();
// Now abort the meta region server and wait for it to go down and come back
stopOrAbortMetaRegionServer(false);
// Verify that everything is back up.
LOG.info("Starting up the verification thread for " + getName());
Thread t = startVerificationThread(tableName, row);
t.start();
threadDumpingJoin(t);
}
private byte [] createTableAndAddRow(final String tableName)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
// put some values in the table
this.table = new HTable(conf, tableName);
byte [] row = Bytes.toBytes("row1");
Put put = new Put(row);
put.add(HConstants.CATALOG_FAMILY, null, Bytes.toBytes(tableName));
table.put(put);
return row;
}
/*
* Stop the region server serving the meta region and wait for the meta region
* to get reassigned. This is always the most problematic case.
*
* @param abort set to true if region server should be aborted, if false it
* is just shut down.
*/
private void stopOrAbortMetaRegionServer(boolean abort) {
List<JVMClusterUtil.RegionServerThread> regionThreads =
cluster.getRegionServerThreads();
int server = -1;
for (int i = 0; i < regionThreads.size() && server == -1; i++) {
HRegionServer s = regionThreads.get(i).getRegionServer();
Collection<HRegion> regions = s.getOnlineRegionsLocalContext();
for (HRegion r : regions) {
if (Bytes.equals(r.getTableDesc().getName(),
HConstants.META_TABLE_NAME)) {
server = i;
}
}
}
if (server == -1) {
LOG.fatal("could not find region server serving meta region");
fail();
}
if (abort) {
this.cluster.abortRegionServer(server);
} else {
this.cluster.stopRegionServer(server);
}
LOG.info(this.cluster.waitOnRegionServer(server) + " has been " +
(abort ? "aborted" : "shut down"));
}
/*
* Run verification in a thread so I can concurrently run a thread-dumper
* while we're waiting (because in this test sometimes the meta scanner
* looks to be be stuck).
* @param tableName Name of table to find.
* @param row Row we expect to find.
* @return Verification thread. Caller needs to calls start on it.
*/
private Thread startVerificationThread(final String tableName,
final byte [] row) {
Runnable runnable = new Runnable() {
public void run() {
try {
// Now try to open a scanner on the meta table. Should stall until
// meta server comes back up.
HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
Scan scan = new Scan();
scan.addFamily(HConstants.CATALOG_FAMILY);
ResultScanner s = t.getScanner(scan);
s.close();
} catch (IOException e) {
LOG.fatal("could not re-open meta table because", e);
fail();
}
ResultScanner scanner = null;
try {
// Verify that the client can find the data after the region has moved
// to a different server
Scan scan = new Scan();
scan.addFamily(HConstants.CATALOG_FAMILY);
scanner = table.getScanner(scan);
LOG.info("Obtained scanner " + scanner);
for (Result r : scanner) {
assertTrue(Bytes.equals(r.getRow(), row));
assertEquals(1, r.size());
byte[] bytes = r.value();
assertNotNull(bytes);
assertTrue(tableName.equals(Bytes.toString(bytes)));
}
LOG.info("Success!");
} catch (Exception e) {
e.printStackTrace();
fail();
} finally {
if (scanner != null) {
LOG.info("Closing scanner " + scanner);
scanner.close();
}
}
}
};
return new Thread(runnable);
}
}

View File

@ -19,24 +19,31 @@
*/
package org.apache.hadoop.hbase.thrift;
import static org.junit.Assert.*;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.Mutation;
import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Unit testing for ThriftServer.HBaseHandler, a part of the
* org.apache.hadoop.hbase.thrift package.
*/
public class TestThriftServer extends HBaseClusterTestCase {
public class TestThriftServer {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
protected static final int MAXVERSIONS = 3;
private static ByteBuffer $bb(String i) {
return ByteBuffer.wrap(Bytes.toBytes(i));
}
@ -52,6 +59,16 @@ public class TestThriftServer extends HBaseClusterTestCase {
private static ByteBuffer valueCname = $bb("valueC");
private static ByteBuffer valueDname = $bb("valueD");
@BeforeClass
public static void beforeClass() throws Exception {
UTIL.startMiniCluster();
}
@AfterClass
public static void afterClass() throws IOException {
UTIL.shutdownMiniCluster();
}
/**
* Runs all of the tests under a single JUnit test method. We
* consolidate all testing to one method because HBaseClusterTestCase
@ -76,8 +93,10 @@ public class TestThriftServer extends HBaseClusterTestCase {
*
* @throws Exception
*/
@Test
public void doTestTableCreateDrop() throws Exception {
ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(this.conf);
ThriftServer.HBaseHandler handler =
new ThriftServer.HBaseHandler(UTIL.getConfiguration());
// Create/enable/disable/delete tables, ensure methods act correctly
assertEquals(handler.getTableNames().size(), 0);
@ -109,7 +128,8 @@ public class TestThriftServer extends HBaseClusterTestCase {
*/
public void doTestTableMutations() throws Exception {
// Setup
ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(this.conf);
ThriftServer.HBaseHandler handler =
new ThriftServer.HBaseHandler(UTIL.getConfiguration());
handler.createTable(tableAname, getColumnDescriptors());
// Apply a few Mutations to rowA
@ -173,7 +193,8 @@ public class TestThriftServer extends HBaseClusterTestCase {
*/
public void doTestTableTimestampsAndColumns() throws Exception {
// Setup
ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(this.conf);
ThriftServer.HBaseHandler handler =
new ThriftServer.HBaseHandler(UTIL.getConfiguration());
handler.createTable(tableAname, getColumnDescriptors());
// Apply timestamped Mutations to rowA
@ -251,7 +272,8 @@ public class TestThriftServer extends HBaseClusterTestCase {
*/
public void doTestTableScanners() throws Exception {
// Setup
ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(this.conf);
ThriftServer.HBaseHandler handler =
new ThriftServer.HBaseHandler(UTIL.getConfiguration());
handler.createTable(tableAname, getColumnDescriptors());
// Apply timestamped Mutations to rowA
@ -318,7 +340,8 @@ public class TestThriftServer extends HBaseClusterTestCase {
* @throws Exception
*/
public void doTestGetTableRegions() throws Exception {
ThriftServer.HBaseHandler handler = new ThriftServer.HBaseHandler(this.conf);
ThriftServer.HBaseHandler handler =
new ThriftServer.HBaseHandler(UTIL.getConfiguration());
handler.createTable(tableAname, getColumnDescriptors());
int regionCount = handler.getTableRegions(tableAname).size();
assertEquals("empty table should have only 1 region, " +