HBASE-4703 Improvements in tests

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1195833 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-11-01 05:36:12 +00:00
parent e29a6c1b3b
commit 9018d028e1
66 changed files with 245 additions and 160 deletions

View File

@ -700,7 +700,7 @@ Release 0.92.0 - Unreleased
HBASE-4694 Some cleanup of log messages in RS and M
HBASE-4603 Uneeded sleep time for tests in
hbase.master.ServerManager#waitForRegionServers (nkeywal)
HBASE-4703 Improvements in tests (nkeywal)
TASKS
HBASE-3559 Move report of split to master OFF the heartbeat channel

View File

@ -770,7 +770,6 @@
<jetty.jspapi.version>6.1.14</jetty.jspapi.version>
<jersey.version>1.4</jersey.version>
<jruby.version>1.6.0</jruby.version>
<jsr311.version>1.1.1</jsr311.version>
<junit.version>4.8.2</junit.version>
<log4j.version>1.2.16</log4j.version>
<mockito-all.version>1.8.5</mockito-all.version>
@ -1035,11 +1034,6 @@
<artifactId>jersey-server</artifactId>
<version>${jersey.version}</version>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>${jsr311.version}</version>
</dependency>
<dependency>
<groupId>javax.xml.bind</groupId>
<artifactId>jaxb-api</artifactId>

View File

@ -1286,12 +1286,14 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
new HDFSBlocksDistribution();
long totalStaticIndexSize = 0;
long totalStaticBloomSize = 0;
long totalMslabWaste = 0;
long tmpfiles;
long tmpindex;
long tmpfilesize;
long tmpbloomsize;
long tmpstaticsize;
long tmpMslabWaste;
String cfname;
// Note that this is a map of Doubles instead of Longs. This is because we
@ -1315,6 +1317,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
tmpfilesize = store.getStorefilesSize();
tmpbloomsize = store.getTotalStaticBloomSize();
tmpstaticsize = store.getTotalStaticIndexSize();
tmpMslabWaste = store.memstore.getMslabWaste();
// Note that there is only one store per CF so setting is safe
cfname = "cf." + store.toString();
@ -1329,11 +1332,14 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
(store.getMemStoreSize() / (1024.0 * 1024)));
this.incrMap(tempVals, cfname + ".staticIndexSizeKB",
tmpstaticsize / 1024.0);
this.incrMap(tempVals, cfname + ".mslabWasteKB",
tmpMslabWaste / 1024.0);
storefiles += tmpfiles;
storefileIndexSize += tmpindex;
totalStaticIndexSize += tmpstaticsize;
totalStaticBloomSize += tmpbloomsize;
totalMslabWaste += tmpMslabWaste;
}
}
@ -1353,6 +1359,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
(int) (totalStaticIndexSize / 1024));
this.metrics.totalStaticBloomSizeKB.set(
(int) (totalStaticBloomSize / 1024));
this.metrics.totalMslabWasteKB.set(
(int) (totalMslabWaste / 1024));
this.metrics.readRequestsCount.set(readRequestsCount);
this.metrics.writeRequestsCount.set(writeRequestsCount);

View File

@ -124,6 +124,18 @@ public class MemStore implements HeapSize {
}
}
/**
* @return the number of bytes "wasted" by external fragmentation
* in the MSLAB, if configured.
*/
long getMslabWaste() {
if (allocator != null) {
return allocator.getWastedBytes();
} else {
return 0;
}
}
void dump() {
for (KeyValue kv: this.kvset) {
LOG.info(kv);

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.regionserver;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.conf.Configuration;
@ -56,7 +57,9 @@ public class MemStoreLAB {
final static String MAX_ALLOC_KEY = "hbase.hregion.memstore.mslab.max.allocation";
final static int MAX_ALLOC_DEFAULT = 256 * 1024; // allocs bigger than this don't go through allocator
final int maxAlloc;
private final AtomicLong wastedSpace = new AtomicLong();
public MemStoreLAB() {
this(new Configuration());
}
@ -103,22 +106,35 @@ public class MemStoreLAB {
}
}
public long getWastedBytes() {
Chunk cur = curChunk.get();
long ret = wastedSpace.get();
if (cur != null) {
ret += cur.getFreeSpace();
}
return ret;
}
/**
* Try to retire the current chunk if it is still
* <code>c</code>. Postcondition is that curChunk.get()
* != c
*/
private void tryRetireChunk(Chunk c) {
@SuppressWarnings("unused")
boolean weRetiredIt = curChunk.compareAndSet(c, null);
// If the CAS succeeds, that means that we won the race
// to retire the chunk. We could use this opportunity to
// update metrics on external fragmentation.
//
// to retire the chunk.
// If the CAS fails, that means that someone else already
// retired the chunk for us.
if (weRetiredIt) {
// This isn't quite right, since another thread may
// have a small allocation concurrently with our retiring
// the chunk. But it should be very close to right,
// and this is just for metrics.
wastedSpace.addAndGet(c.getFreeSpace());
}
}
/**
* Get the current chunk, or, if there is no current chunk,
* allocate a new one from the JVM.
@ -239,6 +255,15 @@ public class MemStoreLAB {
" allocs=" + allocCount.get() + "waste=" +
(data.length - nextFreeOffset.get());
}
private int getFreeSpace() {
int off = nextFreeOffset.get();
if (off >= 0) {
return data.length - off;
} else {
return 0;
}
}
}
/**

View File

@ -155,6 +155,9 @@ public class RegionServerMetrics implements Updater {
public final MetricsIntValue totalStaticBloomSizeKB =
new MetricsIntValue("totalStaticBloomSizeKB", registry);
/** Total amount of memory wasted by external fragmentation in MSLABs */
public final MetricsIntValue totalMslabWasteKB =
new MetricsIntValue("totalMslabWasteKB", registry);
/**
* HDFS blocks locality index
*/

View File

@ -251,13 +251,12 @@ public class HBaseTestingUtility {
}
String randomStr = UUID.randomUUID().toString();
Path testDir= new Path(
Path testPath= new Path(
getBaseTestDir(),
randomStr
);
dataTestDir = new File(testDir.toString()).getAbsoluteFile();
// Have it cleaned up on exit
dataTestDir = new File(testPath.toString()).getAbsoluteFile();
dataTestDir.deleteOnExit();
}
@ -400,7 +399,6 @@ public class HBaseTestingUtility {
throws Exception {
File zkClusterFile = new File(getClusterTestDir().toString());
return startMiniZKCluster(zkClusterFile, zooKeeperServerNum);
}
private MiniZooKeeperCluster startMiniZKCluster(final File dir)
@ -1389,7 +1387,7 @@ public class HBaseTestingUtility {
while (!admin.isTableAvailable(table)) {
assertTrue("Timed out waiting for table " + Bytes.toStringBinary(table),
System.currentTimeMillis() - startWait < timeoutMillis);
Thread.sleep(500);
Thread.sleep(200);
}
}
@ -1402,13 +1400,14 @@ public class HBaseTestingUtility {
*/
public boolean ensureSomeRegionServersAvailable(final int num)
throws IOException {
if (this.getHBaseCluster().getLiveRegionServerThreads().size() < num) {
// Need at least "num" servers.
LOG.info("Started new server=" +
this.getHBaseCluster().startRegionServer());
return true;
boolean startedServer = false;
for (int i=hbaseCluster.getLiveRegionServerThreads().size(); i<num; ++i){
LOG.info("Started new server=" + hbaseCluster.startRegionServer());
startedServer = true;
}
return false;
return startedServer;
}
@ -1503,7 +1502,7 @@ public class HBaseTestingUtility {
break;
}
LOG.info("Found=" + rows);
Threads.sleep(1000);
Threads.sleep(200);
}
}

View File

@ -376,7 +376,7 @@ public class MiniHBaseCluster {
*/
public boolean waitForActiveAndReadyMaster() throws InterruptedException {
List<JVMClusterUtil.MasterThread> mts;
while ((mts = getMasterThreads()).size() > 0) {
while (!(mts = getMasterThreads()).isEmpty()) {
for (JVMClusterUtil.MasterThread mt : mts) {
if (mt.getMaster().isActiveMaster() && mt.getMaster().isInitialized()) {
return true;

View File

@ -99,6 +99,8 @@ public class TestGlobalMemStoreSize {
assertEquals(server.getRegionServerAccounting().getGlobalMemstoreSize(),
0);
}
TEST_UTIL.shutdownMiniCluster();
}
/** figure out how many regions are currently being served. */
@ -128,7 +130,7 @@ public class TestGlobalMemStoreSize {
while (getRegionCount() < totalRegionNum) {
LOG.debug("Waiting for there to be "+totalRegionNum+" regions, but there are " + getRegionCount() + " right now.");
try {
Thread.sleep(1000);
Thread.sleep(100);
} catch (InterruptedException e) {}
}
}

View File

@ -146,6 +146,7 @@ public class TestSerialization {
byte[] val = "val".getBytes();
Put put = new Put(row);
put.setWriteToWAL(false);
put.add(fam, qf1, ts, val);
put.add(fam, qf2, ts, val);
put.add(fam, qf3, ts, val);

View File

@ -242,6 +242,7 @@ public class TimestampTestBase extends HBaseTestCase {
final long ts)
throws IOException {
Put put = new Put(ROW, ts, null);
put.setWriteToWAL(false);
put.add(FAMILY_NAME, QUALIFIER_NAME, bytes);
loader.put(put);
}

View File

@ -62,7 +62,7 @@ public class TestAvroServer {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
}
/**

View File

@ -325,7 +325,7 @@ public class TestAdmin {
while (!done.get()) {
synchronized (done) {
try {
done.wait(1000);
done.wait(100);
} catch (InterruptedException e) {
e.printStackTrace();
}
@ -678,6 +678,7 @@ public class TestAdmin {
for (int i = 0; i < rowCounts[index]; i++) {
byte[] k = Bytes.toBytes(i);
Put put = new Put(k);
put.setWriteToWAL(false);
put.add(familyNames[index], new byte[0], k);
table.put(put);
}
@ -867,7 +868,7 @@ public class TestAdmin {
for (int i = 0; i < count; i++) {
while(threads[i].isAlive()) {
try {
Thread.sleep(1000);
Thread.sleep(100);
} catch (InterruptedException e) {
// continue
}
@ -1067,10 +1068,16 @@ public class TestAdmin {
}
}
}
Thread.sleep(1000);
onlineRegions = rs.getOnlineRegions();
boolean isInList = rs.getOnlineRegions().contains(info);
long timeout = System.currentTimeMillis() + 2000;
while ((System.currentTimeMillis() < timeout) && (isInList)) {
Thread.sleep(100);
isInList = rs.getOnlineRegions().contains(info);
}
assertFalse("The region should not be present in online regions list.",
onlineRegions.contains(info));
isInList);
}
@Test
@ -1305,6 +1312,7 @@ public class TestAdmin {
.toBytes(tableName));
for (int i = 1; i <= 256; i++) { // 256 writes should cause 8 log rolls
Put put = new Put(Bytes.toBytes("row" + String.format("%1$04d", i)));
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, null, value);
table.put(put);
if (i % 32 == 0) {

View File

@ -99,6 +99,7 @@ public class TestFromClientSide {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// We need more than one region server in this test
TEST_UTIL.startMiniCluster(3);
}
@ -480,6 +481,7 @@ public class TestFromClientSide {
System.out.println(String.format("Saving row: %s, with value %s", row,
value));
Put put = new Put(Bytes.toBytes(row));
put.setWriteToWAL(false);
put.add(Bytes.toBytes("trans-blob"), null, Bytes
.toBytes("value for blob"));
put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
@ -656,7 +658,6 @@ public class TestFromClientSide {
Result result = scanner.next();
assertTrue("Expected null result", result == null);
scanner.close();
System.out.println("Done.");
}
@Test
@ -695,6 +696,7 @@ public class TestFromClientSide {
};
for(int i=0;i<10;i++) {
Put put = new Put(ROWS[i]);
put.setWriteToWAL(false);
put.add(FAMILY, QUALIFIERS[i], VALUE);
ht.put(put);
}
@ -730,6 +732,7 @@ public class TestFromClientSide {
};
for(int i=0;i<10;i++) {
Put put = new Put(ROWS[i]);
put.setWriteToWAL(false);
put.add(FAMILY, QUALIFIERS[i], VALUE);
ht.put(put);
}
@ -1954,6 +1957,7 @@ public class TestFromClientSide {
for (int i = 0; i < 10; i++) {
byte [] bytes = Bytes.toBytes(i);
put = new Put(bytes);
put.setWriteToWAL(false);
put.add(FAMILIES[0], QUALIFIER, bytes);
ht.put(put);
}
@ -2061,6 +2065,7 @@ public class TestFromClientSide {
for(int i=0;i<numRows;i++) {
Put put = new Put(ROWS[i]);
put.setWriteToWAL(false);
for(int j=0;j<numColsPerRow;j++) {
put.add(FAMILY, QUALIFIERS[j], QUALIFIERS[j]);
}
@ -3607,6 +3612,7 @@ public class TestFromClientSide {
for (int i = 0; i < NB_BATCH_ROWS; i++) {
byte[] row = Bytes.toBytes("row" + i);
Put put = new Put(row);
put.setWriteToWAL(false);
put.add(CONTENTS_FAMILY, null, value);
rowsUpdate.add(put);
}
@ -3634,6 +3640,7 @@ public class TestFromClientSide {
for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
byte[] row = Bytes.toBytes("row" + i);
Put put = new Put(row);
put.setWriteToWAL(false);
put.add(CONTENTS_FAMILY, null, value);
rowsUpdate.add(put);
}
@ -3675,6 +3682,7 @@ public class TestFromClientSide {
for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
byte[] row = Bytes.toBytes("row" + i);
Put put = new Put(row);
put.setWriteToWAL(false);
put.add(CONTENTS_FAMILY, null, value);
rowsUpdate.add(put);
}
@ -3870,6 +3878,7 @@ public class TestFromClientSide {
try {
for (Result r : s) {
put = new Put(r.getRow());
put.setWriteToWAL(false);
for (KeyValue kv : r.raw()) {
put.add(kv);
}

View File

@ -49,7 +49,7 @@ public class TestHTableUtil {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
}
/**

View File

@ -298,6 +298,7 @@ public class TestMetaMigrationRemovingHTD {
HRegionInfo090x hri = new HRegionInfo090x(htd,
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);
@ -341,6 +342,7 @@ public class TestMetaMigrationRemovingHTD {
HRegionInfo hri = new HRegionInfo(htd.getName(),
startKeys[i], startKeys[j]);
Put put = new Put(hri.getRegionName());
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
meta.put(put);

View File

@ -58,8 +58,8 @@ public class TestMultiParallel {
UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY));
}
@AfterClass public static void afterClass() throws IOException {
UTIL.getMiniHBaseCluster().shutdown();
@AfterClass public static void afterClass() throws Exception {
UTIL.shutdownMiniCluster();
}
@Before public void before() throws IOException {
@ -472,7 +472,6 @@ public class TestMultiParallel {
private void validateLoadedData(HTable table) throws IOException {
// get the data back and validate that it is correct
for (byte[] k : KEYS) {
LOG.info("Assert=" + Bytes.toString(k));
Get get = new Get(k);
get.addColumn(BYTES_FAMILY, QUALIFIER);
Result r = table.get(get);

View File

@ -51,7 +51,7 @@ public class TestMultipleTimestamps {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
}
/**
@ -451,6 +451,7 @@ public class TestMultipleTimestamps {
for (int rowIdx: rowIndexes) {
byte row[] = Bytes.toBytes("row:" + rowIdx);
Put put = new Put(row);
put.setWriteToWAL(false);
for(int colIdx: columnIndexes) {
byte column[] = Bytes.toBytes("column:" + colIdx);
for (long version: versions) {
@ -472,6 +473,7 @@ public class TestMultipleTimestamps {
byte row[] = Bytes.toBytes("row:" + rowIdx);
byte column[] = Bytes.toBytes("column:" + colIdx);
Put put = new Put(row);
put.setWriteToWAL(false);
for (long idx = versionStart; idx <= versionEnd; idx++) {
put.add(cf, column, idx, Bytes.toBytes("value-version-" + idx));

View File

@ -59,6 +59,7 @@ public class TestScannerTimeout {
public static void setUpBeforeClass() throws Exception {
Configuration c = TEST_UTIL.getConfiguration();
c.setInt("hbase.regionserver.lease.period", SCANNER_TIMEOUT);
// We need more than one region server for this test
TEST_UTIL.startMiniCluster(2);
HTable table = TEST_UTIL.createTable(TABLE_NAME, SOME_BYTES);
for (int i = 0; i < NB_ROWS; i++) {

View File

@ -53,7 +53,7 @@ public class TestTimestampsFilter {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
}
/**
@ -363,6 +363,7 @@ public class TestTimestampsFilter {
byte row[] = Bytes.toBytes("row:" + rowIdx);
byte column[] = Bytes.toBytes("column:" + colIdx);
Put put = new Put(row);
put.setWriteToWAL(false);
for (long idx = versionStart; idx <= versionEnd; idx++) {
put.add(cf, column, idx, Bytes.toBytes("value-version-" + idx));

View File

@ -87,10 +87,12 @@ public class TestAggregateProtocol {
*/
for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]);
put.setWriteToWAL(false);
Long l = new Long(i);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(l));
table.put(put);
Put p2 = new Put(ROWS[i]);
put.setWriteToWAL(false);
p2.add(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(l)), Bytes
.toBytes(l * 10));
table.put(p2);

View File

@ -75,12 +75,17 @@ public class TestCoprocessorEndpoint {
for (int i = 0; i < ROWSIZE; i++) {
Put put = new Put(ROWS[i]);
put.setWriteToWAL(false);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i));
table.put(put);
}
// sleep here is an ugly hack to allow region transitions to finish
Thread.sleep(5000);
long timeout = System.currentTimeMillis() + (15 * 1000);
while ((System.currentTimeMillis() < timeout) &&
(table.getRegionsInfo().size() != 2)) {
Thread.sleep(250);
}
}
@AfterClass

View File

@ -138,7 +138,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
BuggyMasterObserver.class.getName());
conf.set("hbase.coprocessor.abortonerror", "true");
UTIL.startMiniCluster(2);
UTIL.startMiniCluster();
}
@AfterClass

View File

@ -120,7 +120,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
Configuration conf = UTIL.getConfiguration();
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
BuggyMasterObserver.class.getName());
UTIL.startMiniCluster(2);
UTIL.startMiniCluster();
}
@AfterClass

View File

@ -477,7 +477,7 @@ public class TestMasterObserver {
Configuration conf = UTIL.getConfiguration();
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
CPMasterObserver.class.getName());
// We need more than one data server on this test
UTIL.startMiniCluster(2);
}

View File

@ -72,7 +72,7 @@ public class TestRegionObserverInterface {
conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
"org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver");
util.startMiniCluster(2);
util.startMiniCluster();
cluster = util.getMiniHBaseCluster();
}
@ -327,9 +327,10 @@ public class TestRegionObserverInterface {
HTable table = new HTable(util.getConfiguration(), compactTable);
for (long i=1; i<=10; i++) {
byte[] iBytes = Bytes.toBytes(i);
Put p = new Put(iBytes);
p.add(A, A, iBytes);
table.put(p);
Put put = new Put(iBytes);
put.setWriteToWAL(false);
put.add(A, A, iBytes);
table.put(put);
}
HRegion firstRegion = cluster.getRegions(compactTable).get(0);

View File

@ -70,6 +70,7 @@ public class TestColumnPrefixFilter {
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
@ -126,6 +127,7 @@ public class TestColumnPrefixFilter {
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,

View File

@ -129,7 +129,7 @@ public class TestColumnRangeFilter {
*/
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
}
/**
@ -183,6 +183,7 @@ public class TestColumnRangeFilter {
for (String row : rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column : columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
@ -200,11 +201,6 @@ public class TestColumnRangeFilter {
}
TEST_UTIL.flush();
try {
Thread.sleep(3000);
} catch (InterruptedException i) {
// ignore
}
ColumnRangeFilter filter;
Scan scan = new Scan();

View File

@ -98,6 +98,7 @@ public class TestFilter extends HBaseTestCase {
// Insert first half
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
}
@ -105,6 +106,7 @@ public class TestFilter extends HBaseTestCase {
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
}
@ -117,6 +119,7 @@ public class TestFilter extends HBaseTestCase {
// Insert second half (reverse families)
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
}
@ -124,6 +127,7 @@ public class TestFilter extends HBaseTestCase {
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
}
@ -1182,7 +1186,9 @@ public class TestFilter extends HBaseTestCase {
};
for(KeyValue kv : srcKVs) {
this.region.put(new Put(kv.getRow()).add(kv));
Put put = new Put(kv.getRow()).add(kv);
put.setWriteToWAL(false);
this.region.put(put);
}
// Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false

View File

@ -72,6 +72,7 @@ public class TestMultipleColumnPrefixFilter {
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
@ -131,6 +132,7 @@ public class TestMultipleColumnPrefixFilter {
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
double rand = Math.random();
@ -185,6 +187,7 @@ public class TestMultipleColumnPrefixFilter {
for (String row: rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column: columns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,

View File

@ -53,7 +53,8 @@ import org.junit.Before;
import org.junit.Test;
public class TestHFileBlock {
// change this value to activate more logs
private static final boolean detailedLogging = false;
private static final boolean[] BOOLEAN_VALUES = new boolean[] { false, true };
private static final Log LOG = LogFactory.getLog(TestHFileBlock.class);
@ -250,7 +251,7 @@ public class TestHFileBlock {
List<ByteBuffer> expectedContents = cacheOnWrite
? new ArrayList<ByteBuffer>() : null;
long totalSize = writeBlocks(rand, algo, path, expectedOffsets,
expectedPrevOffsets, expectedTypes, expectedContents, true);
expectedPrevOffsets, expectedTypes, expectedContents);
FSDataInputStream is = fs.open(path);
HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, algo,
@ -263,10 +264,13 @@ public class TestHFileBlock {
}
assertEquals(expectedOffsets.get(i).longValue(), curOffset);
LOG.info("Reading block #" + i + " at offset " + curOffset);
if (detailedLogging) {
LOG.info("Reading block #" + i + " at offset " + curOffset);
}
HFileBlock b = hbr.readBlockData(curOffset, -1, -1, pread);
LOG.info("Block #" + i + ": " + b);
if (detailedLogging) {
LOG.info("Block #" + i + ": " + b);
}
assertEquals("Invalid block #" + i + "'s type:",
expectedTypes.get(i), b.getBlockType());
assertEquals("Invalid previous block offset for block " + i
@ -388,8 +392,9 @@ public class TestHFileBlock {
++numWithOnDiskSize;
}
LOG.info("Client " + clientId + " successfully read " + numBlocksRead +
" blocks (with pread: " + numPositionalRead + ", with onDiskSize " +
"specified: " + numWithOnDiskSize + ")");
" blocks (with pread: " + numPositionalRead + ", with onDiskSize " +
"specified: " + numWithOnDiskSize + ")");
return true;
}
@ -403,7 +408,7 @@ public class TestHFileBlock {
Random rand = defaultRandom();
List<Long> offsets = new ArrayList<Long>();
List<BlockType> types = new ArrayList<BlockType>();
writeBlocks(rand, compressAlgo, path, offsets, null, types, null, false);
writeBlocks(rand, compressAlgo, path, offsets, null, types, null);
FSDataInputStream is = fs.open(path);
long fileSize = fs.getFileStatus(path).getLen();
HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, compressAlgo,
@ -421,9 +426,11 @@ public class TestHFileBlock {
for (int i = 0; i < NUM_READER_THREADS; ++i) {
Future<Boolean> result = ecs.take();
assertTrue(result.get());
LOG.info(String.valueOf(i + 1)
if (detailedLogging) {
LOG.info(String.valueOf(i + 1)
+ " reader threads finished successfully (algo=" + compressAlgo
+ ")");
}
}
is.close();
@ -432,8 +439,8 @@ public class TestHFileBlock {
private long writeBlocks(Random rand, Compression.Algorithm compressAlgo,
Path path, List<Long> expectedOffsets, List<Long> expectedPrevOffsets,
List<BlockType> expectedTypes, List<ByteBuffer> expectedContents,
boolean detailedLogging) throws IOException {
List<BlockType> expectedTypes, List<ByteBuffer> expectedContents
) throws IOException {
boolean cacheOnWrite = expectedContents != null;
FSDataOutputStream os = fs.create(path);
HFileBlock.Writer hbw = new HFileBlock.Writer(compressAlgo);

View File

@ -53,9 +53,8 @@ public class TestLruBlockCache extends TestCase {
// Let the eviction run
int n = 0;
while(cache.getEvictionCount() == 0) {
System.out.println("sleep");
Thread.sleep(1000);
assertTrue(n++ < 2);
Thread.sleep(200);
assertTrue(n++ < 10);
}
System.out.println("Background Evictions run: " + cache.getEvictionCount());

View File

@ -73,7 +73,6 @@ public class TestReseekTo {
String value = valueList.get(i);
long start = System.nanoTime();
scanner.seekTo(Bytes.toBytes(key));
System.out.println("Seek Finished in: " + (System.nanoTime() - start)/1000 + " micro s");
assertEquals(value, scanner.getValueString());
}
@ -83,7 +82,6 @@ public class TestReseekTo {
String value = valueList.get(i);
long start = System.nanoTime();
scanner.reseekTo(Bytes.toBytes(key));
System.out.println("Reseek Finished in: " + (System.nanoTime() - start)/1000 + " micro s");
assertEquals(value, scanner.getValueString());
}
}

View File

@ -67,12 +67,12 @@ public class TestTableInputFormat {
@BeforeClass
public static void beforeClass() throws Exception {
UTIL.startMiniCluster(1);
UTIL.startMiniCluster();
}
@AfterClass
public static void afterClass() throws IOException {
UTIL.getMiniHBaseCluster().shutdown();
public static void afterClass() throws Exception {
UTIL.shutdownMiniCluster();
}
@Before

View File

@ -35,6 +35,7 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -410,7 +411,7 @@ public class TestHFileOutputFormat {
admin.disableTable(table.getTableName());
while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
isRegionsInTransition()) {
Threads.sleep(1000);
Threads.sleep(200);
LOG.info("Waiting on table to finish disabling");
}
byte[][] newStartKeys = generateRandomStartKeys(15);
@ -419,7 +420,7 @@ public class TestHFileOutputFormat {
admin.enableTable(table.getTableName());
while (table.getRegionsInfo().size() != 15 ||
!admin.isTableAvailable(table.getTableName())) {
Thread.sleep(1000);
Thread.sleep(200);
LOG.info("Waiting for new region assignment to happen");
}
}
@ -449,7 +450,7 @@ public class TestHFileOutputFormat {
// Cause regions to reopen
admin.disableTable(TABLE_NAME);
while (!admin.isTableDisabled(TABLE_NAME)) {
Thread.sleep(1000);
Thread.sleep(200);
LOG.info("Waiting for table to disable");
}
admin.enableTable(TABLE_NAME);
@ -469,9 +470,11 @@ public class TestHFileOutputFormat {
setupRandomGeneratorMapper(job);
HFileOutputFormat.configureIncrementalLoad(job, table);
FileOutputFormat.setOutputPath(job, outDir);
Assert.assertFalse( util.getTestFileSystem().exists(outDir)) ;
assertEquals(table.getRegionsInfo().size(),
job.getNumReduceTasks());
job.getNumReduceTasks());
assertTrue(job.waitForCompletion(true));
}

View File

@ -86,20 +86,6 @@ public class TestTableInputFormatScan {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setUp() throws Exception {
// nothing
}
/**
* @throws java.lang.Exception
*/
@After
public void tearDown() throws Exception {
Configuration c = TEST_UTIL.getConfiguration();
FileUtil.fullyDelete(new File(c.get("hadoop.tmp.dir")));
}
/**
* Pass the key and value to reduce.
*/

View File

@ -124,6 +124,7 @@ public class TestTimeRangeMapRed {
for (Long ts : tsList) {
Put put = new Put(key.get());
put.setWriteToWAL(false);
put.add(FAMILY_NAME, COLUMN_NAME, ts, Bytes.toBytes(true));
table.put(put);
}
@ -163,6 +164,7 @@ public class TestTimeRangeMapRed {
private void prepareTest(final HTable table) throws IOException {
for (Map.Entry<Long, Boolean> entry : TIMESTAMP.entrySet()) {
Put put = new Put(KEY);
put.setWriteToWAL(false);
put.add(FAMILY_NAME, COLUMN_NAME, entry.getKey(), Bytes.toBytes(false));
table.put(put);
}

View File

@ -56,6 +56,7 @@ public class TsvImporterCustomTestMapper extends TsvImporterMapper {
ImmutableBytesWritable rowKey =
new ImmutableBytesWritable(Bytes.toBytes(valueTokens[0]));
Put put = new Put(rowKey.copyBytes());
put.setWriteToWAL(false);
//The value should look like this: VALUE1 or VALUE2. Let's multiply
//the integer by 3

View File

@ -489,6 +489,7 @@ public class TestMasterTransitions {
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);
p.setWriteToWAL(false);
p.add(getTestFamily(), getTestQualifier(), row);
t.put(p);
rows++;

View File

@ -360,6 +360,7 @@ public class TestZKBasedOpenCloseRegion {
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);
p.setWriteToWAL(false);
p.add(getTestFamily(), getTestQualifier(), row);
t.put(p);
rows++;

View File

@ -51,7 +51,6 @@ public class TestMemoryBoundedLogMessageBuffer {
StringWriter sw = new StringWriter();
buf.dumpTo(new PrintWriter(sw));
String dump = sw.toString();
System.out.println(dump);
assertFalse("The early log messages should be evicted",
dump.contains("hello 1\n"));
assertTrue("The late log messages should be retained",

View File

@ -122,6 +122,7 @@ public class TestBlocksRead extends HBaseTestCase {
long versionEnd) throws IOException {
byte columnBytes[] = Bytes.toBytes(col);
Put put = new Put(Bytes.toBytes(row));
put.setWriteToWAL(false);
for (long version = versionStart; version <= versionEnd; version++) {
put.add(cf, columnBytes, version, genValue(row, col, version));

View File

@ -101,6 +101,7 @@ public class TestColumnSeeking {
for (String value : values) {
for (String row : rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column : allColumns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv =
@ -205,6 +206,7 @@ public class TestColumnSeeking {
for (String row : rows) {
Put p = new Put(Bytes.toBytes(row));
p.setWriteToWAL(false);
for (String column : allColumns) {
for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
KeyValue kv =

View File

@ -394,6 +394,7 @@ public class TestCompaction extends HBaseTestCase {
for (int i = 0; i < compactionThreshold; i++) {
HRegionIncommon loader = new HRegionIncommon(r);
Put p = new Put(Bytes.add(STARTROW, Bytes.toBytes(i)));
p.setWriteToWAL(false);
for (int j = 0; j < jmax; j++) {
p.add(COLUMN_FAMILY, Bytes.toBytes(j), pad);
}

View File

@ -78,6 +78,7 @@ public class TestGetClosestAtOrBefore extends HBaseTestCase {
i == 0? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i),
i == last? HConstants.EMPTY_BYTE_ARRAY: Bytes.toBytes((byte)i + interval));
Put put = new Put(hri.getRegionName());
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
Writables.getBytes(hri));
mr.put(put, false);

View File

@ -319,6 +319,7 @@ public class TestHRegion extends HBaseTestCase {
System.out.println(String.format("Saving row: %s, with value %s", row,
value));
Put put = new Put(Bytes.toBytes(row));
put.setWriteToWAL(false);
put.add(Bytes.toBytes("trans-blob"), null,
Bytes.toBytes("value for blob"));
put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
@ -2452,6 +2453,7 @@ public class TestHRegion extends HBaseTestCase {
boolean toggle=true;
for (long i = 0; i < numRows; i++) {
Put put = new Put(Bytes.toBytes(i));
put.setWriteToWAL(false);
put.add(family, qual1, Bytes.toBytes(i % 10));
region.put(put);
@ -2662,6 +2664,7 @@ public class TestHRegion extends HBaseTestCase {
for (int r = 0; r < numRows; r++) {
byte[] row = Bytes.toBytes("row" + r);
Put put = new Put(row);
put.setWriteToWAL(false);
byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
for (byte[] family : families) {
for (byte[] qualifier : qualifiers) {
@ -2879,6 +2882,7 @@ public class TestHRegion extends HBaseTestCase {
for (int i = 0; i < duplicate_multiplier; i ++) {
for (int j = 0; j < num_unique_rows; j++) {
Put put = new Put(Bytes.toBytes("row" + j));
put.setWriteToWAL(false);
put.add(fam1, qf1, version++, val1);
region.put(put);
}
@ -2932,6 +2936,7 @@ public class TestHRegion extends HBaseTestCase {
byte row[] = Bytes.toBytes("row:" + 0);
byte column[] = Bytes.toBytes("column:" + 0);
Put put = new Put(row);
put.setWriteToWAL(false);
for (long idx = 1; idx <= 4; idx++) {
put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
}
@ -3057,6 +3062,7 @@ public class TestHRegion extends HBaseTestCase {
throws IOException {
for(int i=startRow; i<startRow+numRows; i++) {
Put put = new Put(Bytes.toBytes("" + i));
put.setWriteToWAL(false);
for(byte [] family : families) {
put.add(family, qf, null);
}

View File

@ -46,7 +46,9 @@ public class TestMemStoreLAB {
public void testLABRandomAllocation() {
Random rand = new Random();
MemStoreLAB mslab = new MemStoreLAB();
assertEquals(0, mslab.getWastedBytes());
int expectedOff = 0;
int slabsUsed = 0;
byte[] lastBuffer = null;
// 100K iterations by 0-1K alloc -> 50MB expected
// should be reasonable for unit test and also cover wraparound
@ -58,12 +60,21 @@ public class TestMemStoreLAB {
if (alloc.getData() != lastBuffer) {
expectedOff = 0;
lastBuffer = alloc.getData();
slabsUsed++;
}
assertEquals(expectedOff, alloc.getOffset());
assertTrue("Allocation " + alloc + " overruns buffer",
alloc.getOffset() + size <= alloc.getData().length);
expectedOff += size;
}
// maximum waste is 1KB per slab plus
// whatever's left in current slab
long expectedWaste = slabsUsed * 1000 +
(lastBuffer.length - expectedOff);
long waste = mslab.getWastedBytes();
assertTrue("waste should be less than " + expectedWaste +
" but was: " + waste, waste < expectedWaste);
}
@Test

View File

@ -229,9 +229,11 @@ public class TestSeekOptimizations {
+ columnRestrictionStr + ", " + rowRestrictionStr
+ ", maxVersions=" + maxVersions + ", lazySeek=" + lazySeekEnabled;
long seekCount = StoreFileScanner.getSeekCount() - initialSeekCount;
System.err.println("Seek count: " + seekCount + ", KVs returned: "
if (VERBOSE) {
System.err.println("Seek count: " + seekCount + ", KVs returned: "
+ actualKVs.size() + ". " + testDesc +
(lazySeekEnabled ? "\n" : ""));
}
if (lazySeekEnabled) {
totalSeekLazy += seekCount;
} else {

View File

@ -81,6 +81,7 @@ public class TestWideScanner extends HBaseTestCase {
byte[] b = Bytes.toBytes(String.format("%10d", i));
for (j = 0; j < 100; j++) {
Put put = new Put(row);
put.setWriteToWAL(false);
put.add(COLUMNS[rng.nextInt(COLUMNS.length)], b, ++ts, b);
region.put(put);
count++;

View File

@ -48,7 +48,6 @@ import org.junit.Test;
public class TestReplicationSink {
private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
private static final int BATCH_SIZE = 10;
private static final long SLEEP_TIME = 500;
private final static HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
@ -112,7 +111,6 @@ public class TestReplicationSink {
public void setUp() throws Exception {
table1 = TEST_UTIL.truncateTable(TABLE_NAME1);
table2 = TEST_UTIL.truncateTable(TABLE_NAME2);
Thread.sleep(SLEEP_TIME);
}
/**

View File

@ -59,7 +59,7 @@ public class TestGzipFilter {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
client = new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort()));

View File

@ -71,7 +71,7 @@ public class TestMultiRowResource {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(conf);
context = JAXBContext.newInstance(
CellModel.class,

View File

@ -88,6 +88,7 @@ public class TestScannerResource {
k[1] = b2;
k[2] = b3;
Put put = new Put(k);
put.setWriteToWAL(false);
put.add(famAndQf[0], famAndQf[1], k);
table.put(put);
count++;
@ -149,7 +150,7 @@ public class TestScannerResource {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(conf);
client = new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort()));

View File

@ -139,6 +139,7 @@ public class TestScannersWithFilters {
// Insert first half
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
}
@ -146,6 +147,7 @@ public class TestScannersWithFilters {
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
}
@ -155,6 +157,7 @@ public class TestScannersWithFilters {
// Insert second half (reverse families)
for(byte [] ROW : ROWS_ONE) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_ONE) {
p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
}
@ -162,6 +165,7 @@ public class TestScannersWithFilters {
}
for(byte [] ROW : ROWS_TWO) {
Put p = new Put(ROW);
p.setWriteToWAL(false);
for(byte [] QUALIFIER : QUALIFIERS_TWO) {
p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
}

View File

@ -57,7 +57,7 @@ public class TestSchemaResource {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(conf);
client = new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort()));

View File

@ -75,7 +75,7 @@ public class TestStatusResource {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
client = new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort()));

View File

@ -97,6 +97,7 @@ public class TestTableResource {
k[1] = b2;
k[2] = b3;
Put put = new Put(k);
put.setWriteToWAL(false);
put.add(famAndQf[0], famAndQf[1], k);
table.put(put);
}
@ -109,13 +110,18 @@ public class TestTableResource {
// tell the master to split the table
admin.split(TABLE);
// give some time for the split to happen
try {
Thread.sleep(15 * 1000);
} catch (InterruptedException e) {
LOG.warn(StringUtils.stringifyException(e));
long timeout = System.currentTimeMillis() + (15 * 1000);
while (System.currentTimeMillis() < timeout && m.size()!=2){
try {
Thread.sleep(250);
} catch (InterruptedException e) {
LOG.warn(StringUtils.stringifyException(e));
}
// check again
m = table.getRegionsInfo();
}
// check again
m = table.getRegionsInfo();
// should have two regions now
assertEquals(m.size(), 2);
regionMap = m;

View File

@ -55,7 +55,7 @@ public class TestTransform {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
client = new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort()));

View File

@ -54,7 +54,7 @@ public class TestVersionResource {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
client = new Client(new Cluster().add("localhost",
REST_TEST_UTIL.getServletPort()));

View File

@ -34,43 +34,25 @@ import org.junit.BeforeClass;
import org.junit.Test;
public class TestRemoteAdmin {
private static final String TABLE_1 = "TestRemoteAdmin_Table_1";
private static final String TABLE_2 = "TestRemoteAdmin_Table_2";
private static final byte[] COLUMN_1 = Bytes.toBytes("a");
static final HTableDescriptor DESC_1;
static {
DESC_1 = new HTableDescriptor(TABLE_1);
DESC_1.addFamily(new HColumnDescriptor(COLUMN_1));
}
static final HTableDescriptor DESC_2;
static {
DESC_2 = new HTableDescriptor(TABLE_2);
DESC_2.addFamily(new HColumnDescriptor(COLUMN_1));
}
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
static final HTableDescriptor DESC_1 = new HTableDescriptor(TABLE_1);
private static final HBaseTestingUtility TEST_UTIL =
new HBaseTestingUtility();
private static final HBaseRESTTestingUtility REST_TEST_UTIL =
new HBaseRESTTestingUtility();
private static HBaseAdmin localAdmin;
private static RemoteAdmin remoteAdmin;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
DESC_1.addFamily(new HColumnDescriptor(COLUMN_1));
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
localAdmin = TEST_UTIL.getHBaseAdmin();
remoteAdmin = new RemoteAdmin(new Client(
new Cluster().add("localhost", REST_TEST_UTIL.getServletPort())),
TEST_UTIL.getConfiguration());
if (localAdmin.tableExists(TABLE_1)) {
localAdmin.disableTable(TABLE_1);
localAdmin.deleteTable(TABLE_1);
}
if (!localAdmin.tableExists(TABLE_2)) {
localAdmin.createTable(DESC_2);
}
}
@AfterClass
@ -80,16 +62,11 @@ public class TestRemoteAdmin {
}
@Test
public void testCreateTable() throws Exception {
public void testCreateAnDeleteTable() throws Exception {
assertFalse(remoteAdmin.isTableAvailable(TABLE_1));
remoteAdmin.createTable(DESC_1);
assertTrue(remoteAdmin.isTableAvailable(TABLE_1));
}
@Test
public void testDeleteTable() throws Exception {
assertTrue(remoteAdmin.isTableAvailable(TABLE_2));
remoteAdmin.deleteTable(TABLE_2);
assertFalse(remoteAdmin.isTableAvailable(TABLE_2));
remoteAdmin.deleteTable(TABLE_1);
assertFalse(remoteAdmin.isTableAvailable(TABLE_1));
}
}

View File

@ -75,7 +75,7 @@ public class TestRemoteTable {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster();
REST_TEST_UTIL.startServletContainer(TEST_UTIL.getConfiguration());
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
LOG.info("Admin Connection=" + admin.getConnection() + ", " +

View File

@ -122,6 +122,7 @@ public class TestHBaseFsck {
// When we find a diff RS, change the assignment and break
if (startCode != sn.getStartcode()) {
Put put = new Put(res.getRow());
put.setWriteToWAL(false);
put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort()));
put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
@ -135,13 +136,12 @@ public class TestHBaseFsck {
// Try to fix the data
assertErrors(doFsck(true), new ERROR_CODE[]{
ERROR_CODE.SERVER_DOES_NOT_MATCH_META});
Thread.sleep(15000);
// Should be fixed now
assertNoErrors(doFsck(false));
// comment needed - what is the purpose of this line
new HTable(conf, Bytes.toBytes(table)).getScanner(new Scan());;
new HTable(conf, Bytes.toBytes(table)).getScanner(new Scan());
}
private HRegionInfo createRegion(Configuration conf, final HTableDescriptor

View File

@ -41,7 +41,7 @@ public class TestIdLock {
private static final int NUM_IDS = 16;
private static final int NUM_THREADS = 128;
private static final int NUM_SECONDS = 20;
private static final int NUM_SECONDS = 15;
private IdLock idLock = new IdLock();
@ -63,7 +63,6 @@ public class TestIdLock {
while (System.currentTimeMillis() < endTime) {
long id = rand.nextInt(NUM_IDS);
LOG.info(clientId + " is waiting for id " + id);
IdLock.Entry lockEntry = idLock.getLockEntry(id);
try {
int sleepMs = 1 + rand.nextInt(4);
@ -75,10 +74,7 @@ public class TestIdLock {
}
idOwner.put(id, clientId);
LOG.info(clientId + " took id " + id + ", sleeping for " +
sleepMs + "ms");
Thread.sleep(sleepMs);
LOG.info(clientId + " is releasing id " + id);
idOwner.remove(id);
} finally {

View File

@ -143,6 +143,7 @@ public class TestMergeTable {
LOG.info("Created region " + region.getRegionNameAsString());
for(int i = firstRow; i < firstRow + nrows; i++) {
Put put = new Put(Bytes.toBytes("row_" + String.format("%1$05d", i)));
put.setWriteToWAL(false);
put.add(COLUMN_NAME, null, VALUE);
region.put(put);
if (i % 10000 == 0) {

View File

@ -27,12 +27,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
@ -46,6 +41,7 @@ import org.apache.hadoop.util.ToolRunner;
/** Test stand alone merge tool that can merge arbitrary regions */
public class TestMergeTool extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestMergeTool.class);
HBaseTestingUtility TEST_UTIL;
// static final byte [] COLUMN_NAME = Bytes.toBytes("contents:");
static final byte [] FAMILY = Bytes.toBytes("contents");
static final byte [] QUALIFIER = Bytes.toBytes("dc");
@ -123,7 +119,8 @@ public class TestMergeTool extends HBaseTestCase {
"row_1000", "row_1000", "row_1000", "row_1000" });
// Start up dfs
this.dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
TEST_UTIL = new HBaseTestingUtility(conf);
this.dfsCluster = TEST_UTIL.startMiniDFSCluster(2);
this.fs = this.dfsCluster.getFileSystem();
System.out.println("fs=" + this.fs);
this.conf.set("fs.defaultFS", fs.getUri().toString());
@ -162,7 +159,7 @@ public class TestMergeTool extends HBaseTestCase {
closeRootAndMeta();
} catch (Exception e) {
shutdownDfs(dfsCluster);
TEST_UTIL.shutdownMiniCluster();
throw e;
}
}
@ -170,7 +167,7 @@ public class TestMergeTool extends HBaseTestCase {
@Override
public void tearDown() throws Exception {
super.tearDown();
shutdownDfs(dfsCluster);
TEST_UTIL.shutdownMiniCluster();
}
/*

View File

@ -287,6 +287,7 @@ public class TestRegionSplitter {
for(byte b=Byte.MIN_VALUE; b<Byte.MAX_VALUE; b++) {
byte[] whateverBytes = new byte[] {b};
Put p = new Put(whateverBytes);
p.setWriteToWAL(false);
p.add(CF_NAME.getBytes(), whateverBytes, whateverBytes);
hTable.put(p);
}