HBASE-469 Streamline HStore startup and compactions
HMerge, HRegionServer - changes that reflect changes to HRegion, CompactSplitThread and Flusher methods ServerManager - Return zero length array to region server if it is exiting or quiesced and Master is not yet ready to shut down. QueueEntry - removed. no longer used. CompactSplitThread - make compactionQueue a queue of HRegion. - Add Set<HRegion> so we can quickly determine if a region is in the queue. BlockingQueue.contains() does a linear scan of the queue. - Add a lock and interruptPolitely methods so that compactions/splits in progress are not interrupted. - Don't add a region to the queue if it is already present. Flusher - change queue from DelayQueue to BlockingQueue, with HRegion entries instead of QueueEntry. - Add Set<HRegion> to quickly determine if a region is already in the queue to avoid linear scan of BlockingQueue.contains(). - Only put regions in the queue for optional cache flush if the last time they were flushed is older than now - optionalFlushInterval. - Only add regions to the queue if it is not already present. HRegion - don't request a cache flush if one has already been requested. - Add setLastFlushTime so flusher can set it once it has queued an optional flush. - Replace largestHStore with getLargestHStoreSize: returns long instead of HStoreSize object. - Add midKey as parameter to splitRegion. - Reorder start of splitRegion so it doesn't do any work before validating parameters. - Remove needsSplit and compactIfNeeded - no longer needed. - compactStores now returns midKey if split is needed. - snapshotMemcaches now sets flushRequested to false and sets lastFlushTime to now. - update does not request a cache flush if one has already been requested. - Override equals and hashCode so HRegions can be stored in a HashSet. HStore - loadHStoreFiles now computes max sequence id and the initial size of the store. - Add getter for family. - internalCacheFlush updates store size, and logs both size of cache flush and resulting map file size (with debug logging enabled). - Remove needsCompaction and hasReferences - no longer needed. - compact() returns midKey if store needs to be split. - compact() does all checking before actually starting a compaction. - If store size is greater than desiredMaxFileSize, compact returns the midKey for the store regardless of whether a compaction was actually done. - Added more synchronization in completeCompaction while iterating over storeFiles. - completeCompaction computes new store size. - New method checkSplit replaces method size. Returns midKey if store needs to be split and can be split. HStoreSize - removed. No longer needed. HBaseTestCase - only set fs if it has not already been set by a subclass. TestTableIndex, TestTableMapReduce - call FSUtil.deleteFully to clean up cruft left in local fs, by MapReduce git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@643761 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0e825ebaff
commit
6e225dd7f1
|
@ -12,6 +12,9 @@ Hbase Change Log
|
||||||
NEW FEATURES
|
NEW FEATURES
|
||||||
HBASE-548 Tool to online single region
|
HBASE-548 Tool to online single region
|
||||||
|
|
||||||
|
IMPROVEMENTS
|
||||||
|
HBASE-469 Streamline HStore startup and compactions
|
||||||
|
|
||||||
Release 0.1.0
|
Release 0.1.0
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -144,17 +144,16 @@ class HMerge implements HConstants {
|
||||||
long currentSize = 0;
|
long currentSize = 0;
|
||||||
HRegion nextRegion = null;
|
HRegion nextRegion = null;
|
||||||
long nextSize = 0;
|
long nextSize = 0;
|
||||||
Text midKey = new Text();
|
|
||||||
for (int i = 0; i < info.length - 1; i++) {
|
for (int i = 0; i < info.length - 1; i++) {
|
||||||
if (currentRegion == null) {
|
if (currentRegion == null) {
|
||||||
currentRegion =
|
currentRegion =
|
||||||
new HRegion(tabledir, hlog, fs, conf, info[i], null, null);
|
new HRegion(tabledir, hlog, fs, conf, info[i], null, null);
|
||||||
currentSize = currentRegion.largestHStore(midKey).getAggregate();
|
currentSize = currentRegion.getLargestHStoreSize();
|
||||||
}
|
}
|
||||||
nextRegion =
|
nextRegion =
|
||||||
new HRegion(tabledir, hlog, fs, conf, info[i + 1], null, null);
|
new HRegion(tabledir, hlog, fs, conf, info[i + 1], null, null);
|
||||||
|
|
||||||
nextSize = nextRegion.largestHStore(midKey).getAggregate();
|
nextSize = nextRegion.getLargestHStoreSize();
|
||||||
|
|
||||||
if ((currentSize + nextSize) <= (maxFilesize / 2)) {
|
if ((currentSize + nextSize) <= (maxFilesize / 2)) {
|
||||||
// We merge two adjacent regions if their total size is less than
|
// We merge two adjacent regions if their total size is less than
|
||||||
|
|
|
@ -128,6 +128,9 @@ class ServerManager implements HConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Called to process the messages sent from the region server to the master
|
||||||
|
* along with the heart beat.
|
||||||
|
*
|
||||||
* @param serverInfo
|
* @param serverInfo
|
||||||
* @param msgs
|
* @param msgs
|
||||||
* @return messages from master to region server indicating what region
|
* @return messages from master to region server indicating what region
|
||||||
|
@ -142,7 +145,7 @@ class ServerManager implements HConstants {
|
||||||
if (msgs.length > 0) {
|
if (msgs.length > 0) {
|
||||||
if (msgs[0].getMsg() == HMsg.MSG_REPORT_EXITING) {
|
if (msgs[0].getMsg() == HMsg.MSG_REPORT_EXITING) {
|
||||||
processRegionServerExit(serverName, msgs);
|
processRegionServerExit(serverName, msgs);
|
||||||
return new HMsg[]{msgs[0]};
|
return new HMsg[0];
|
||||||
} else if (msgs[0].getMsg() == HMsg.MSG_REPORT_QUIESCED) {
|
} else if (msgs[0].getMsg() == HMsg.MSG_REPORT_QUIESCED) {
|
||||||
LOG.info("Region server " + serverName + " quiesced");
|
LOG.info("Region server " + serverName + " quiesced");
|
||||||
master.quiescedMetaServers.incrementAndGet();
|
master.quiescedMetaServers.incrementAndGet();
|
||||||
|
@ -157,6 +160,11 @@ class ServerManager implements HConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (master.shutdownRequested && !master.closed.get()) {
|
if (master.shutdownRequested && !master.closed.get()) {
|
||||||
|
if (msgs.length > 0 && msgs[0].getMsg() == HMsg.MSG_REPORT_QUIESCED) {
|
||||||
|
// Server is already quiesced, but we aren't ready to shut down
|
||||||
|
// return empty response
|
||||||
|
return new HMsg[0];
|
||||||
|
}
|
||||||
// Tell the server to stop serving any user regions
|
// Tell the server to stop serving any user regions
|
||||||
return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE)};
|
return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE)};
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -45,16 +46,19 @@ implements RegionUnavailableListener, HConstants {
|
||||||
|
|
||||||
private HTable root = null;
|
private HTable root = null;
|
||||||
private HTable meta = null;
|
private HTable meta = null;
|
||||||
private long startTime;
|
private volatile long startTime;
|
||||||
private final long frequency;
|
private final long frequency;
|
||||||
|
private final Integer lock = new Integer(0);
|
||||||
|
|
||||||
private HRegionServer server;
|
private final HRegionServer server;
|
||||||
private HBaseConfiguration conf;
|
private final HBaseConfiguration conf;
|
||||||
|
|
||||||
private final BlockingQueue<QueueEntry> compactionQueue =
|
private final BlockingQueue<HRegion> compactionQueue =
|
||||||
new LinkedBlockingQueue<QueueEntry>();
|
new LinkedBlockingQueue<HRegion>();
|
||||||
|
|
||||||
/** constructor */
|
private final HashSet<HRegion> regionsInQueue = new HashSet<HRegion>();
|
||||||
|
|
||||||
|
/** @param server */
|
||||||
public CompactSplitThread(HRegionServer server) {
|
public CompactSplitThread(HRegionServer server) {
|
||||||
super();
|
super();
|
||||||
this.server = server;
|
this.server = server;
|
||||||
|
@ -68,19 +72,26 @@ implements RegionUnavailableListener, HConstants {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
while (!server.isStopRequested()) {
|
while (!server.isStopRequested()) {
|
||||||
QueueEntry e = null;
|
HRegion r = null;
|
||||||
try {
|
try {
|
||||||
e = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS);
|
r = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS);
|
||||||
if (e == null) {
|
if (r != null) {
|
||||||
continue;
|
synchronized (regionsInQueue) {
|
||||||
|
regionsInQueue.remove(r);
|
||||||
|
}
|
||||||
|
synchronized (lock) {
|
||||||
|
// Don't interrupt us while we are working
|
||||||
|
Text midKey = r.compactStores();
|
||||||
|
if (midKey != null) {
|
||||||
|
split(r, midKey);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
e.getRegion().compactIfNeeded();
|
|
||||||
split(e.getRegion());
|
|
||||||
} catch (InterruptedException ex) {
|
} catch (InterruptedException ex) {
|
||||||
continue;
|
continue;
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
LOG.error("Compaction failed" +
|
LOG.error("Compaction failed" +
|
||||||
(e != null ? (" for region " + e.getRegion().getRegionName()) : ""),
|
(r != null ? (" for region " + r.getRegionName()) : ""),
|
||||||
RemoteExceptionHandler.checkIOException(ex));
|
RemoteExceptionHandler.checkIOException(ex));
|
||||||
if (!server.checkFileSystem()) {
|
if (!server.checkFileSystem()) {
|
||||||
break;
|
break;
|
||||||
|
@ -88,30 +99,35 @@ implements RegionUnavailableListener, HConstants {
|
||||||
|
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
LOG.error("Compaction failed" +
|
LOG.error("Compaction failed" +
|
||||||
(e != null ? (" for region " + e.getRegion().getRegionName()) : ""),
|
(r != null ? (" for region " + r.getRegionName()) : ""),
|
||||||
ex);
|
ex);
|
||||||
if (!server.checkFileSystem()) {
|
if (!server.checkFileSystem()) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
regionsInQueue.clear();
|
||||||
|
compactionQueue.clear();
|
||||||
LOG.info(getName() + " exiting");
|
LOG.info(getName() + " exiting");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param e QueueEntry for region to be compacted
|
* @param r HRegion store belongs to
|
||||||
*/
|
*/
|
||||||
public void compactionRequested(QueueEntry e) {
|
public synchronized void compactionRequested(HRegion r) {
|
||||||
compactionQueue.add(e);
|
LOG.debug("Compaction requested for region: " + r.getRegionName());
|
||||||
|
synchronized (regionsInQueue) {
|
||||||
|
if (!regionsInQueue.contains(r)) {
|
||||||
|
compactionQueue.add(r);
|
||||||
|
regionsInQueue.add(r);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void compactionRequested(final HRegion r) {
|
private void split(final HRegion region, final Text midKey)
|
||||||
compactionRequested(new QueueEntry(r, System.currentTimeMillis()));
|
throws IOException {
|
||||||
}
|
|
||||||
|
|
||||||
private void split(final HRegion region) throws IOException {
|
|
||||||
final HRegionInfo oldRegionInfo = region.getRegionInfo();
|
final HRegionInfo oldRegionInfo = region.getRegionInfo();
|
||||||
final HRegion[] newRegions = region.splitRegion(this);
|
final HRegion[] newRegions = region.splitRegion(this, midKey);
|
||||||
if (newRegions == null) {
|
if (newRegions == null) {
|
||||||
// Didn't need to be split
|
// Didn't need to be split
|
||||||
return;
|
return;
|
||||||
|
@ -198,4 +214,13 @@ implements RegionUnavailableListener, HConstants {
|
||||||
server.getWriteLock().unlock();
|
server.getWriteLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Only interrupt once it's done with a run through the work loop.
|
||||||
|
*/
|
||||||
|
void interruptPolitely() {
|
||||||
|
synchronized (lock) {
|
||||||
|
interrupt();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,14 +20,16 @@
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.concurrent.DelayQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.ConcurrentModificationException;
|
import java.util.ConcurrentModificationException;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
||||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||||
|
@ -35,60 +37,60 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||||
/** Flush cache upon request */
|
/** Flush cache upon request */
|
||||||
class Flusher extends Thread implements CacheFlushListener {
|
class Flusher extends Thread implements CacheFlushListener {
|
||||||
static final Log LOG = LogFactory.getLog(Flusher.class);
|
static final Log LOG = LogFactory.getLog(Flusher.class);
|
||||||
private final DelayQueue<QueueEntry> flushQueue =
|
private final BlockingQueue<HRegion> flushQueue =
|
||||||
new DelayQueue<QueueEntry>();
|
new LinkedBlockingQueue<HRegion>();
|
||||||
|
|
||||||
|
private final HashSet<HRegion> regionsInQueue = new HashSet<HRegion>();
|
||||||
|
|
||||||
|
private final long threadWakeFrequency;
|
||||||
private final long optionalFlushPeriod;
|
private final long optionalFlushPeriod;
|
||||||
private final HRegionServer server;
|
private final HRegionServer server;
|
||||||
private final HBaseConfiguration conf;
|
|
||||||
private final Integer lock = new Integer(0);
|
private final Integer lock = new Integer(0);
|
||||||
|
|
||||||
/** constructor */
|
/**
|
||||||
public Flusher(final HRegionServer server) {
|
* @param conf
|
||||||
|
* @param server
|
||||||
|
*/
|
||||||
|
public Flusher(final HBaseConfiguration conf, final HRegionServer server) {
|
||||||
super();
|
super();
|
||||||
this.server = server;
|
this.server = server;
|
||||||
conf = server.conf;
|
|
||||||
this.optionalFlushPeriod = conf.getLong(
|
this.optionalFlushPeriod = conf.getLong(
|
||||||
"hbase.regionserver.optionalcacheflushinterval", 30 * 60 * 1000L);
|
"hbase.regionserver.optionalcacheflushinterval", 30 * 60 * 1000L);
|
||||||
|
this.threadWakeFrequency = conf.getLong(
|
||||||
|
HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
|
long lastOptionalCheck = System.currentTimeMillis();
|
||||||
while (!server.isStopRequested()) {
|
while (!server.isStopRequested()) {
|
||||||
QueueEntry e = null;
|
HRegion r = null;
|
||||||
try {
|
try {
|
||||||
e = flushQueue.poll(server.threadWakeFrequency, TimeUnit.MILLISECONDS);
|
long now = System.currentTimeMillis();
|
||||||
if (e == null) {
|
if (now - threadWakeFrequency > lastOptionalCheck) {
|
||||||
continue;
|
lastOptionalCheck = now;
|
||||||
|
// Queue up regions for optional flush if they need it
|
||||||
|
Set<HRegion> regions = server.getRegionsToCheck();
|
||||||
|
for (HRegion region: regions) {
|
||||||
|
synchronized (regionsInQueue) {
|
||||||
|
if (!regionsInQueue.contains(region) &&
|
||||||
|
(now - optionalFlushPeriod) > region.getLastFlushTime()) {
|
||||||
|
regionsInQueue.add(region);
|
||||||
|
flushQueue.add(region);
|
||||||
|
region.setLastFlushTime(now);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
r = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
|
||||||
|
if (r != null) {
|
||||||
|
synchronized (regionsInQueue) {
|
||||||
|
regionsInQueue.remove(r);
|
||||||
}
|
}
|
||||||
synchronized (lock) { // Don't interrupt while we're working
|
synchronized (lock) { // Don't interrupt while we're working
|
||||||
if (e.getRegion().flushcache()) {
|
if (r.flushcache()) {
|
||||||
server.compactionRequested(e);
|
server.compactSplitThread.compactionRequested(r);
|
||||||
}
|
|
||||||
|
|
||||||
e.setExpirationTime(System.currentTimeMillis() +
|
|
||||||
optionalFlushPeriod);
|
|
||||||
flushQueue.add(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Now ensure that all the active regions are in the queue
|
|
||||||
Set<HRegion> regions = server.getRegionsToCheck();
|
|
||||||
for (HRegion r: regions) {
|
|
||||||
e = new QueueEntry(r, r.getLastFlushTime() + optionalFlushPeriod);
|
|
||||||
synchronized (flushQueue) {
|
|
||||||
if (!flushQueue.contains(e)) {
|
|
||||||
flushQueue.add(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Now make sure that the queue only contains active regions
|
|
||||||
synchronized (flushQueue) {
|
|
||||||
for (Iterator<QueueEntry> i = flushQueue.iterator(); i.hasNext(); ) {
|
|
||||||
e = i.next();
|
|
||||||
if (!regions.contains(e.getRegion())) {
|
|
||||||
i.remove();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -108,32 +110,32 @@ class Flusher extends Thread implements CacheFlushListener {
|
||||||
server.stop();
|
server.stop();
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
LOG.error("Cache flush failed" +
|
LOG.error("Cache flush failed" +
|
||||||
(e != null ? (" for region " + e.getRegion().getRegionName()) : ""),
|
(r != null ? (" for region " + r.getRegionName()) : ""),
|
||||||
RemoteExceptionHandler.checkIOException(ex));
|
RemoteExceptionHandler.checkIOException(ex));
|
||||||
if (!server.checkFileSystem()) {
|
if (!server.checkFileSystem()) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
LOG.error("Cache flush failed" +
|
LOG.error("Cache flush failed" +
|
||||||
(e != null ? (" for region " + e.getRegion().getRegionName()) : ""),
|
(r != null ? (" for region " + r.getRegionName()) : ""),
|
||||||
ex);
|
ex);
|
||||||
if (!server.checkFileSystem()) {
|
if (!server.checkFileSystem()) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
regionsInQueue.clear();
|
||||||
flushQueue.clear();
|
flushQueue.clear();
|
||||||
LOG.info(getName() + " exiting");
|
LOG.info(getName() + " exiting");
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void flushRequested(HRegion region) {
|
public void flushRequested(HRegion r) {
|
||||||
QueueEntry e = new QueueEntry(region, System.currentTimeMillis());
|
synchronized (regionsInQueue) {
|
||||||
synchronized (flushQueue) {
|
if (!regionsInQueue.contains(r)) {
|
||||||
if (flushQueue.contains(e)) {
|
regionsInQueue.add(r);
|
||||||
flushQueue.remove(e);
|
flushQueue.add(r);
|
||||||
}
|
}
|
||||||
flushQueue.add(e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -316,6 +316,7 @@ public class HRegion implements HConstants {
|
||||||
new ConcurrentHashMap<Long, TreeMap<HStoreKey, byte []>>();
|
new ConcurrentHashMap<Long, TreeMap<HStoreKey, byte []>>();
|
||||||
|
|
||||||
final AtomicLong memcacheSize = new AtomicLong(0);
|
final AtomicLong memcacheSize = new AtomicLong(0);
|
||||||
|
private volatile boolean flushRequested;
|
||||||
|
|
||||||
final Path basedir;
|
final Path basedir;
|
||||||
final HLog log;
|
final HLog log;
|
||||||
|
@ -348,7 +349,6 @@ public class HRegion implements HConstants {
|
||||||
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
private final Integer updateLock = new Integer(0);
|
private final Integer updateLock = new Integer(0);
|
||||||
private final Integer splitLock = new Integer(0);
|
private final Integer splitLock = new Integer(0);
|
||||||
private final long desiredMaxFileSize;
|
|
||||||
private final long minSequenceId;
|
private final long minSequenceId;
|
||||||
final AtomicInteger activeScannerCount = new AtomicInteger(0);
|
final AtomicInteger activeScannerCount = new AtomicInteger(0);
|
||||||
|
|
||||||
|
@ -359,6 +359,8 @@ public class HRegion implements HConstants {
|
||||||
/**
|
/**
|
||||||
* HRegion constructor.
|
* HRegion constructor.
|
||||||
*
|
*
|
||||||
|
* @param basedir qualified path of directory where region should be located,
|
||||||
|
* usually the table directory.
|
||||||
* @param log The HLog is the outbound log for any updates to the HRegion
|
* @param log The HLog is the outbound log for any updates to the HRegion
|
||||||
* (There's a single HLog for all the HRegions on a single HRegionServer.)
|
* (There's a single HLog for all the HRegions on a single HRegionServer.)
|
||||||
* The log file is a logfile from the previous execution that's
|
* The log file is a logfile from the previous execution that's
|
||||||
|
@ -366,20 +368,19 @@ public class HRegion implements HConstants {
|
||||||
* appropriate log info for this HRegion. If there is a previous log file
|
* appropriate log info for this HRegion. If there is a previous log file
|
||||||
* (implying that the HRegion has been written-to before), then read it from
|
* (implying that the HRegion has been written-to before), then read it from
|
||||||
* the supplied path.
|
* the supplied path.
|
||||||
* @param basedir qualified path of directory where region should be located,
|
|
||||||
* usually the table directory.
|
|
||||||
* @param fs is the filesystem.
|
* @param fs is the filesystem.
|
||||||
* @param conf is global configuration settings.
|
* @param conf is global configuration settings.
|
||||||
* @param regionInfo - HRegionInfo that describes the region
|
* @param regionInfo - HRegionInfo that describes the region
|
||||||
* @param initialFiles If there are initial files (implying that the HRegion
|
* @param initialFiles If there are initial files (implying that the HRegion
|
||||||
* is new), then read them from the supplied path.
|
* is new), then read them from the supplied path.
|
||||||
* @param listener an object that implements CacheFlushListener or null
|
* @param flushListener an object that implements CacheFlushListener or null
|
||||||
|
* or null
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf,
|
public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf,
|
||||||
HRegionInfo regionInfo, Path initialFiles, CacheFlushListener listener)
|
HRegionInfo regionInfo, Path initialFiles,
|
||||||
throws IOException {
|
CacheFlushListener flushListener) throws IOException {
|
||||||
this(basedir, log, fs, conf, regionInfo, initialFiles, listener, null);
|
this(basedir, log, fs, conf, regionInfo, initialFiles, flushListener, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -399,15 +400,15 @@ public class HRegion implements HConstants {
|
||||||
* @param regionInfo - HRegionInfo that describes the region
|
* @param regionInfo - HRegionInfo that describes the region
|
||||||
* @param initialFiles If there are initial files (implying that the HRegion
|
* @param initialFiles If there are initial files (implying that the HRegion
|
||||||
* is new), then read them from the supplied path.
|
* is new), then read them from the supplied path.
|
||||||
* @param listener an object that implements CacheFlushListener or null
|
* @param flushListener an object that implements CacheFlushListener or null
|
||||||
* @param reporter Call on a period so hosting server can report we're
|
* @param reporter Call on a period so hosting server can report we're
|
||||||
* making progress to master -- otherwise master might think region deploy
|
* making progress to master -- otherwise master might think region deploy
|
||||||
* failed. Can be null.
|
* failed. Can be null.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf,
|
public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf,
|
||||||
HRegionInfo regionInfo, Path initialFiles, CacheFlushListener listener,
|
HRegionInfo regionInfo, Path initialFiles,
|
||||||
final Progressable reporter)
|
CacheFlushListener flushListener, final Progressable reporter)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
this.basedir = basedir;
|
this.basedir = basedir;
|
||||||
|
@ -415,6 +416,8 @@ public class HRegion implements HConstants {
|
||||||
this.fs = fs;
|
this.fs = fs;
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
this.regionInfo = regionInfo;
|
this.regionInfo = regionInfo;
|
||||||
|
this.flushListener = flushListener;
|
||||||
|
this.flushRequested = false;
|
||||||
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
|
this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||||
this.regiondir = new Path(basedir, this.regionInfo.getEncodedName());
|
this.regiondir = new Path(basedir, this.regionInfo.getEncodedName());
|
||||||
Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
|
Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
|
||||||
|
@ -466,14 +469,10 @@ public class HRegion implements HConstants {
|
||||||
// By default, we flush the cache when 64M.
|
// By default, we flush the cache when 64M.
|
||||||
this.memcacheFlushSize = conf.getInt("hbase.hregion.memcache.flush.size",
|
this.memcacheFlushSize = conf.getInt("hbase.hregion.memcache.flush.size",
|
||||||
1024*1024*64);
|
1024*1024*64);
|
||||||
this.flushListener = listener;
|
|
||||||
this.blockingMemcacheSize = this.memcacheFlushSize *
|
this.blockingMemcacheSize = this.memcacheFlushSize *
|
||||||
conf.getInt("hbase.hregion.memcache.block.multiplier", 1);
|
conf.getInt("hbase.hregion.memcache.block.multiplier", 1);
|
||||||
|
|
||||||
// By default we split region if a file > DEFAULT_MAX_FILE_SIZE.
|
|
||||||
this.desiredMaxFileSize =
|
|
||||||
conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE);
|
|
||||||
|
|
||||||
// HRegion is ready to go!
|
// HRegion is ready to go!
|
||||||
this.writestate.compacting = false;
|
this.writestate.compacting = false;
|
||||||
this.lastFlushTime = System.currentTimeMillis();
|
this.lastFlushTime = System.currentTimeMillis();
|
||||||
|
@ -672,6 +671,11 @@ public class HRegion implements HConstants {
|
||||||
return this.lastFlushTime;
|
return this.lastFlushTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** @param t the lastFlushTime */
|
||||||
|
void setLastFlushTime(long t) {
|
||||||
|
this.lastFlushTime = t;
|
||||||
|
}
|
||||||
|
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
// HRegion maintenance.
|
// HRegion maintenance.
|
||||||
//
|
//
|
||||||
|
@ -679,34 +683,16 @@ public class HRegion implements HConstants {
|
||||||
// upkeep.
|
// upkeep.
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
|
|
||||||
/**
|
/** @return returns size of largest HStore. */
|
||||||
* @param midkey
|
public long getLargestHStoreSize() {
|
||||||
* @return returns size of largest HStore. Also returns whether store is
|
long size = 0;
|
||||||
* splitable or not (Its not splitable if region has a store that has a
|
|
||||||
* reference store file).
|
|
||||||
*/
|
|
||||||
public HStoreSize largestHStore(Text midkey) {
|
|
||||||
HStoreSize biggest = null;
|
|
||||||
boolean splitable = true;
|
|
||||||
for (HStore h: stores.values()) {
|
for (HStore h: stores.values()) {
|
||||||
HStoreSize size = h.size(midkey);
|
long storeSize = h.getSize();
|
||||||
// If we came across a reference down in the store, then propagate
|
if (storeSize > size) {
|
||||||
// fact that region is not splitable.
|
size = storeSize;
|
||||||
if (splitable) {
|
|
||||||
splitable = size.splitable;
|
|
||||||
}
|
|
||||||
if (biggest == null) {
|
|
||||||
biggest = size;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
if(size.getAggregate() > biggest.getAggregate()) { // Largest so far
|
|
||||||
biggest = size;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (biggest != null) {
|
return size;
|
||||||
biggest.setSplitable(splitable);
|
|
||||||
}
|
|
||||||
return biggest;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -715,21 +701,17 @@ public class HRegion implements HConstants {
|
||||||
* but instead create new 'reference' store files that read off the top and
|
* but instead create new 'reference' store files that read off the top and
|
||||||
* bottom ranges of parent store files.
|
* bottom ranges of parent store files.
|
||||||
* @param listener May be null.
|
* @param listener May be null.
|
||||||
|
* @param midKey key on which to split region
|
||||||
* @return two brand-new (and open) HRegions or null if a split is not needed
|
* @return two brand-new (and open) HRegions or null if a split is not needed
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
HRegion[] splitRegion(final RegionUnavailableListener listener)
|
HRegion[] splitRegion(final RegionUnavailableListener listener,
|
||||||
throws IOException {
|
final Text midKey) throws IOException {
|
||||||
synchronized (splitLock) {
|
synchronized (splitLock) {
|
||||||
Text midKey = new Text();
|
if (closed.get()) {
|
||||||
if (closed.get() || !needsSplit(midKey)) {
|
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
Path splits = new Path(this.regiondir, SPLITDIR);
|
// Add start/end key checking: hbase-428.
|
||||||
if(!this.fs.exists(splits)) {
|
|
||||||
this.fs.mkdirs(splits);
|
|
||||||
}
|
|
||||||
// Make copies just in case and add start/end key checking: hbase-428.
|
|
||||||
Text startKey = new Text(this.regionInfo.getStartKey());
|
Text startKey = new Text(this.regionInfo.getStartKey());
|
||||||
Text endKey = new Text(this.regionInfo.getEndKey());
|
Text endKey = new Text(this.regionInfo.getEndKey());
|
||||||
if (startKey.equals(midKey)) {
|
if (startKey.equals(midKey)) {
|
||||||
|
@ -740,6 +722,11 @@ public class HRegion implements HConstants {
|
||||||
LOG.debug("Endkey and midkey are same, not splitting");
|
LOG.debug("Endkey and midkey are same, not splitting");
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
LOG.info("Starting split of region " + getRegionName());
|
||||||
|
Path splits = new Path(this.regiondir, SPLITDIR);
|
||||||
|
if(!this.fs.exists(splits)) {
|
||||||
|
this.fs.mkdirs(splits);
|
||||||
|
}
|
||||||
HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
|
HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(),
|
||||||
startKey, midKey);
|
startKey, midKey);
|
||||||
Path dirA = new Path(splits, regionAInfo.getEncodedName());
|
Path dirA = new Path(splits, regionAInfo.getEncodedName());
|
||||||
|
@ -805,71 +792,6 @@ public class HRegion implements HConstants {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
|
||||||
* Iterates through all the HStores and finds the one with the largest
|
|
||||||
* MapFile size. If the size is greater than the (currently hard-coded)
|
|
||||||
* threshold, returns true indicating that the region should be split. The
|
|
||||||
* midKey for the largest MapFile is returned through the midKey parameter.
|
|
||||||
* It is possible for us to rule the region non-splitable even in excess of
|
|
||||||
* configured size. This happens if region contains a reference file. If
|
|
||||||
* a reference file, the region can not be split.
|
|
||||||
*
|
|
||||||
* Note that there is no need to do locking in this method because it calls
|
|
||||||
* largestHStore which does the necessary locking.
|
|
||||||
*
|
|
||||||
* @param midKey midKey of the largest MapFile
|
|
||||||
* @return true if the region should be split. midKey is set by this method.
|
|
||||||
* Check it for a midKey value on return.
|
|
||||||
*/
|
|
||||||
boolean needsSplit(Text midKey) {
|
|
||||||
HStoreSize biggest = largestHStore(midKey);
|
|
||||||
if (biggest == null || midKey.getLength() == 0 ||
|
|
||||||
(midKey.equals(getStartKey()) && midKey.equals(getEndKey())) ) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
boolean split = (biggest.getAggregate() >= this.desiredMaxFileSize);
|
|
||||||
if (split) {
|
|
||||||
if (!biggest.isSplitable()) {
|
|
||||||
LOG.warn("Region " + getRegionName().toString() +
|
|
||||||
" is NOT splitable though its aggregate size is " +
|
|
||||||
StringUtils.humanReadableInt(biggest.getAggregate()) +
|
|
||||||
" and desired size is " +
|
|
||||||
StringUtils.humanReadableInt(this.desiredMaxFileSize));
|
|
||||||
split = false;
|
|
||||||
} else {
|
|
||||||
LOG.info("Splitting " + getRegionName().toString() +
|
|
||||||
" because largest aggregate size is " +
|
|
||||||
StringUtils.humanReadableInt(biggest.getAggregate()) +
|
|
||||||
" and desired size is " +
|
|
||||||
StringUtils.humanReadableInt(this.desiredMaxFileSize));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return split;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Only do a compaction if it is necessary
|
|
||||||
*
|
|
||||||
* @return whether or not there was a compaction
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public boolean compactIfNeeded() throws IOException {
|
|
||||||
boolean needsCompaction = false;
|
|
||||||
for (HStore store: stores.values()) {
|
|
||||||
if (store.needsCompaction()) {
|
|
||||||
needsCompaction = true;
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug(store.toString() + " needs compaction");
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (!needsCompaction) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return compactStores();
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* @param dir
|
* @param dir
|
||||||
* @return compaction directory for the passed in <code>dir</code>
|
* @return compaction directory for the passed in <code>dir</code>
|
||||||
|
@ -893,59 +815,53 @@ public class HRegion implements HConstants {
|
||||||
*/
|
*/
|
||||||
private void doRegionCompactionCleanup() throws IOException {
|
private void doRegionCompactionCleanup() throws IOException {
|
||||||
if (this.fs.exists(this.regionCompactionDir)) {
|
if (this.fs.exists(this.regionCompactionDir)) {
|
||||||
this.fs.delete(this.regionCompactionDir);
|
FileUtil.fullyDelete(this.fs, this.regionCompactionDir);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compact all the stores. This should be called periodically to make sure
|
* Called by compaction thread and after region is opened to compact the
|
||||||
* the stores are kept manageable.
|
* HStores if necessary.
|
||||||
*
|
*
|
||||||
* <p>This operation could block for a long time, so don't call it from a
|
* <p>This operation could block for a long time, so don't call it from a
|
||||||
* time-sensitive thread.
|
* time-sensitive thread.
|
||||||
*
|
*
|
||||||
* @return Returns TRUE if the compaction has completed. FALSE, if the
|
|
||||||
* compaction was not carried out, because the HRegion is busy doing
|
|
||||||
* something else storage-intensive (like flushing the cache). The caller
|
|
||||||
* should check back later.
|
|
||||||
*
|
|
||||||
* Note that no locking is necessary at this level because compaction only
|
* Note that no locking is necessary at this level because compaction only
|
||||||
* conflicts with a region split, and that cannot happen because the region
|
* conflicts with a region split, and that cannot happen because the region
|
||||||
* server does them sequentially and not in parallel.
|
* server does them sequentially and not in parallel.
|
||||||
*
|
*
|
||||||
|
* @return mid key if split is needed
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public boolean compactStores() throws IOException {
|
public Text compactStores() throws IOException {
|
||||||
|
Text midKey = null;
|
||||||
if (this.closed.get()) {
|
if (this.closed.get()) {
|
||||||
return false;
|
return midKey;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
synchronized (writestate) {
|
synchronized (writestate) {
|
||||||
if (!writestate.compacting && writestate.writesEnabled) {
|
if (!writestate.compacting && writestate.writesEnabled) {
|
||||||
writestate.compacting = true;
|
writestate.compacting = true;
|
||||||
} else {
|
} else {
|
||||||
LOG.info("NOT compacting region " +
|
LOG.info("NOT compacting region " + getRegionName() +
|
||||||
this.regionInfo.getRegionName().toString() + ": compacting=" +
|
": compacting=" + writestate.compacting + ", writesEnabled=" +
|
||||||
writestate.compacting + ", writesEnabled=" +
|
|
||||||
writestate.writesEnabled);
|
writestate.writesEnabled);
|
||||||
return false;
|
return midKey;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
LOG.info("starting compaction on region " + getRegionName());
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
LOG.info("starting compaction on region " +
|
|
||||||
this.regionInfo.getRegionName().toString());
|
|
||||||
boolean status = true;
|
|
||||||
doRegionCompactionPrep();
|
doRegionCompactionPrep();
|
||||||
for (HStore store: stores.values()) {
|
for (HStore store: stores.values()) {
|
||||||
if(!store.compact()) {
|
Text key = store.compact();
|
||||||
status = false;
|
if (key != null && midKey == null) {
|
||||||
|
midKey = key;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
doRegionCompactionCleanup();
|
doRegionCompactionCleanup();
|
||||||
LOG.info("compaction completed on region " +
|
LOG.info("compaction completed on region " + getRegionName() +
|
||||||
this.regionInfo.getRegionName().toString() + ". Took " +
|
". Took " +
|
||||||
StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
|
StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
|
||||||
return status;
|
|
||||||
|
|
||||||
} finally {
|
} finally {
|
||||||
synchronized (writestate) {
|
synchronized (writestate) {
|
||||||
|
@ -953,6 +869,7 @@ public class HRegion implements HConstants {
|
||||||
writestate.notifyAll();
|
writestate.notifyAll();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return midKey;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1030,6 +947,10 @@ public class HRegion implements HConstants {
|
||||||
// will add to the unflushed size
|
// will add to the unflushed size
|
||||||
|
|
||||||
this.memcacheSize.set(0L);
|
this.memcacheSize.set(0L);
|
||||||
|
this.flushRequested = false;
|
||||||
|
|
||||||
|
// Record latest flush time
|
||||||
|
this.lastFlushTime = System.currentTimeMillis();
|
||||||
|
|
||||||
for (HStore hstore: stores.values()) {
|
for (HStore hstore: stores.values()) {
|
||||||
hstore.snapshotMemcache();
|
hstore.snapshotMemcache();
|
||||||
|
@ -1121,11 +1042,12 @@ public class HRegion implements HConstants {
|
||||||
this.log.completeCacheFlush(this.regionInfo.getRegionName(),
|
this.log.completeCacheFlush(this.regionInfo.getRegionName(),
|
||||||
regionInfo.getTableDesc().getName(), sequenceId);
|
regionInfo.getTableDesc().getName(), sequenceId);
|
||||||
|
|
||||||
// D. Finally notify anyone waiting on memcache to clear:
|
// C. Finally notify anyone waiting on memcache to clear:
|
||||||
// e.g. checkResources().
|
// e.g. checkResources().
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
notifyAll();
|
notifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Finished memcache flush for region " +
|
LOG.debug("Finished memcache flush for region " +
|
||||||
this.regionInfo.getRegionName() + " in " +
|
this.regionInfo.getRegionName() + " in " +
|
||||||
|
@ -1374,8 +1296,8 @@ public class HRegion implements HConstants {
|
||||||
Text row = b.getRow();
|
Text row = b.getRow();
|
||||||
long lockid = obtainRowLock(row);
|
long lockid = obtainRowLock(row);
|
||||||
|
|
||||||
long commitTime =
|
long commitTime = (b.getTimestamp() == LATEST_TIMESTAMP) ?
|
||||||
(b.getTimestamp() == LATEST_TIMESTAMP) ? System.currentTimeMillis() : b.getTimestamp();
|
System.currentTimeMillis() : b.getTimestamp();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
List<Text> deletes = null;
|
List<Text> deletes = null;
|
||||||
|
@ -1612,9 +1534,11 @@ public class HRegion implements HConstants {
|
||||||
(val == null ? 0 : val.length));
|
(val == null ? 0 : val.length));
|
||||||
stores.get(HStoreKey.extractFamily(key.getColumn())).add(key, val);
|
stores.get(HStoreKey.extractFamily(key.getColumn())).add(key, val);
|
||||||
}
|
}
|
||||||
if (this.flushListener != null && size > this.memcacheFlushSize) {
|
if (this.flushListener != null && !this.flushRequested &&
|
||||||
|
size > this.memcacheFlushSize) {
|
||||||
// Request a cache flush
|
// Request a cache flush
|
||||||
this.flushListener.flushRequested(this);
|
this.flushListener.flushRequested(this);
|
||||||
|
this.flushRequested = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1730,6 +1654,18 @@ public class HRegion implements HConstants {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
return this.hashCode() == ((HRegion)o).hashCode();
|
||||||
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
return this.regionInfo.getRegionName().hashCode();
|
||||||
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
|
@ -2011,8 +1947,7 @@ public class HRegion implements HConstants {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static void removeRegionFromMETA(final HRegionInterface srvr,
|
public static void removeRegionFromMETA(final HRegionInterface srvr,
|
||||||
final Text metaRegionName, final Text regionName)
|
final Text metaRegionName, final Text regionName) throws IOException {
|
||||||
throws IOException {
|
|
||||||
srvr.deleteAll(metaRegionName, regionName, HConstants.LATEST_TIMESTAMP);
|
srvr.deleteAll(metaRegionName, regionName, HConstants.LATEST_TIMESTAMP);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2025,8 +1960,7 @@ public class HRegion implements HConstants {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static void offlineRegionInMETA(final HRegionInterface srvr,
|
public static void offlineRegionInMETA(final HRegionInterface srvr,
|
||||||
final Text metaRegionName, final HRegionInfo info)
|
final Text metaRegionName, final HRegionInfo info) throws IOException {
|
||||||
throws IOException {
|
|
||||||
BatchUpdate b = new BatchUpdate(info.getRegionName());
|
BatchUpdate b = new BatchUpdate(info.getRegionName());
|
||||||
info.setOffline(true);
|
info.setOffline(true);
|
||||||
b.put(COL_REGIONINFO, Writables.getBytes(info));
|
b.put(COL_REGIONINFO, Writables.getBytes(info));
|
||||||
|
|
|
@ -220,7 +220,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
conf.getInt("hbase.master.lease.period", 30 * 1000);
|
conf.getInt("hbase.master.lease.period", 30 * 1000);
|
||||||
|
|
||||||
// Cache flushing thread.
|
// Cache flushing thread.
|
||||||
this.cacheFlusher = new Flusher(this);
|
this.cacheFlusher = new Flusher(conf, this);
|
||||||
|
|
||||||
// Compaction thread
|
// Compaction thread
|
||||||
this.compactSplitThread = new CompactSplitThread(this);
|
this.compactSplitThread = new CompactSplitThread(this);
|
||||||
|
@ -295,6 +295,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
LOG.info("Server quiesced and not serving any regions. " +
|
LOG.info("Server quiesced and not serving any regions. " +
|
||||||
"Starting shutdown");
|
"Starting shutdown");
|
||||||
stopRequested.set(true);
|
stopRequested.set(true);
|
||||||
|
this.outboundMsgs.clear();
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -412,7 +413,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
// Send interrupts to wake up threads if sleeping so they notice shutdown.
|
// Send interrupts to wake up threads if sleeping so they notice shutdown.
|
||||||
// TODO: Should we check they are alive? If OOME could have exited already
|
// TODO: Should we check they are alive? If OOME could have exited already
|
||||||
cacheFlusher.interruptPolitely();
|
cacheFlusher.interruptPolitely();
|
||||||
compactSplitThread.interrupt();
|
compactSplitThread.interruptPolitely();
|
||||||
synchronized (logRollerLock) {
|
synchronized (logRollerLock) {
|
||||||
this.logRoller.interrupt();
|
this.logRoller.interrupt();
|
||||||
}
|
}
|
||||||
|
@ -828,8 +829,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
} finally {
|
} finally {
|
||||||
this.lock.writeLock().unlock();
|
this.lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
reportOpen(regionInfo);
|
|
||||||
}
|
}
|
||||||
|
reportOpen(regionInfo);
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -1228,10 +1229,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
return lock.writeLock();
|
return lock.writeLock();
|
||||||
}
|
}
|
||||||
|
|
||||||
void compactionRequested(QueueEntry e) {
|
|
||||||
compactSplitThread.compactionRequested(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Immutable list of this servers regions.
|
* @return Immutable list of this servers regions.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -24,15 +24,11 @@ import java.io.UnsupportedEncodingException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.SortedMap;
|
import java.util.SortedMap;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
import java.util.SortedSet;
|
|
||||||
import java.util.TreeSet;
|
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
|
@ -44,7 +40,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.filter.RowFilterInterface;
|
import org.apache.hadoop.hbase.filter.RowFilterInterface;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
@ -53,7 +48,6 @@ import org.apache.hadoop.io.MapFile;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
import org.apache.hadoop.io.SequenceFile;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
import org.apache.hadoop.io.WritableComparable;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
import org.apache.hadoop.hbase.BloomFilterDescriptor;
|
import org.apache.hadoop.hbase.BloomFilterDescriptor;
|
||||||
|
@ -69,7 +63,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||||
import org.apache.hadoop.hbase.io.Cell;
|
import org.apache.hadoop.hbase.io.Cell;
|
||||||
import org.apache.hadoop.hbase.io.RowResult;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* HStore maintains a bunch of data files. It is responsible for maintaining
|
* HStore maintains a bunch of data files. It is responsible for maintaining
|
||||||
|
@ -87,7 +80,7 @@ public class HStore implements HConstants {
|
||||||
* If reference, then the regex has more than just one group. Group 1 is
|
* If reference, then the regex has more than just one group. Group 1 is
|
||||||
* this files id. Group 2 the referenced region name, etc.
|
* this files id. Group 2 the referenced region name, etc.
|
||||||
*/
|
*/
|
||||||
private static Pattern REF_NAME_PARSER =
|
private static final Pattern REF_NAME_PARSER =
|
||||||
Pattern.compile("^(\\d+)(?:\\.(.+))?$");
|
Pattern.compile("^(\\d+)(?:\\.(.+))?$");
|
||||||
|
|
||||||
private static final String BLOOMFILTER_FILE_NAME = "filter";
|
private static final String BLOOMFILTER_FILE_NAME = "filter";
|
||||||
|
@ -101,15 +94,16 @@ public class HStore implements HConstants {
|
||||||
private final HBaseConfiguration conf;
|
private final HBaseConfiguration conf;
|
||||||
private final Path filterDir;
|
private final Path filterDir;
|
||||||
final Filter bloomFilter;
|
final Filter bloomFilter;
|
||||||
private final Path compactionDir;
|
|
||||||
|
|
||||||
private final Integer compactLock = new Integer(0);
|
private final long desiredMaxFileSize;
|
||||||
|
private volatile long storeSize;
|
||||||
|
|
||||||
private final Integer flushLock = new Integer(0);
|
private final Integer flushLock = new Integer(0);
|
||||||
|
|
||||||
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||||
final AtomicInteger activeScanners = new AtomicInteger(0);
|
final AtomicInteger activeScanners = new AtomicInteger(0);
|
||||||
|
|
||||||
final String storeName;
|
final Text storeName;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Sorted Map of readers keyed by sequence id (Most recent should be last in
|
* Sorted Map of readers keyed by sequence id (Most recent should be last in
|
||||||
|
@ -125,8 +119,15 @@ public class HStore implements HConstants {
|
||||||
private final SortedMap<Long, MapFile.Reader> readers =
|
private final SortedMap<Long, MapFile.Reader> readers =
|
||||||
new TreeMap<Long, MapFile.Reader>();
|
new TreeMap<Long, MapFile.Reader>();
|
||||||
|
|
||||||
|
// The most-recent log-seq-ID that's present. The most-recent such ID means
|
||||||
|
// we can ignore all log messages up to and including that ID (because they're
|
||||||
|
// already reflected in the TreeMaps).
|
||||||
private volatile long maxSeqId;
|
private volatile long maxSeqId;
|
||||||
|
|
||||||
|
private final Path compactionDir;
|
||||||
|
private final Integer compactLock = new Integer(0);
|
||||||
private final int compactionThreshold;
|
private final int compactionThreshold;
|
||||||
|
|
||||||
private final ReentrantReadWriteLock newScannerLock =
|
private final ReentrantReadWriteLock newScannerLock =
|
||||||
new ReentrantReadWriteLock();
|
new ReentrantReadWriteLock();
|
||||||
|
|
||||||
|
@ -177,7 +178,17 @@ public class HStore implements HConstants {
|
||||||
|
|
||||||
this.compactionDir = HRegion.getCompactionDir(basedir);
|
this.compactionDir = HRegion.getCompactionDir(basedir);
|
||||||
this.storeName =
|
this.storeName =
|
||||||
this.info.getEncodedName() + "/" + this.family.getFamilyName();
|
new Text(this.info.getEncodedName() + "/" + this.family.getFamilyName());
|
||||||
|
|
||||||
|
// By default, we compact if an HStore has more than
|
||||||
|
// MIN_COMMITS_FOR_COMPACTION map files
|
||||||
|
this.compactionThreshold =
|
||||||
|
conf.getInt("hbase.hstore.compactionThreshold", 3);
|
||||||
|
|
||||||
|
// By default we split region if a file > DEFAULT_MAX_FILE_SIZE.
|
||||||
|
this.desiredMaxFileSize =
|
||||||
|
conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE);
|
||||||
|
this.storeSize = 0L;
|
||||||
|
|
||||||
if (family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) {
|
if (family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) {
|
||||||
this.compression = SequenceFile.CompressionType.BLOCK;
|
this.compression = SequenceFile.CompressionType.BLOCK;
|
||||||
|
@ -219,21 +230,10 @@ public class HStore implements HConstants {
|
||||||
// MapFiles are in a reliable state. Every entry in 'mapdir' must have a
|
// MapFiles are in a reliable state. Every entry in 'mapdir' must have a
|
||||||
// corresponding one in 'loginfodir'. Without a corresponding log info
|
// corresponding one in 'loginfodir'. Without a corresponding log info
|
||||||
// file, the entry in 'mapdir' must be deleted.
|
// file, the entry in 'mapdir' must be deleted.
|
||||||
List<HStoreFile> hstoreFiles = loadHStoreFiles(infodir, mapdir);
|
// loadHStoreFiles also computes the max sequence id
|
||||||
for(HStoreFile hsf: hstoreFiles) {
|
this.maxSeqId = -1L;
|
||||||
this.storefiles.put(Long.valueOf(hsf.loadInfo(fs)), hsf);
|
this.storefiles.putAll(loadHStoreFiles(infodir, mapdir));
|
||||||
}
|
|
||||||
|
|
||||||
// Now go through all the HSTORE_LOGINFOFILEs and figure out the
|
|
||||||
// most-recent log-seq-ID that's present. The most-recent such ID means we
|
|
||||||
// can ignore all log messages up to and including that ID (because they're
|
|
||||||
// already reflected in the TreeMaps).
|
|
||||||
//
|
|
||||||
// If the HSTORE_LOGINFOFILE doesn't contain a number, just ignore it. That
|
|
||||||
// means it was built prior to the previous run of HStore, and so it cannot
|
|
||||||
// contain any updates also contained in the log.
|
|
||||||
|
|
||||||
this.maxSeqId = getMaxSequenceId(hstoreFiles);
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("maximum sequence id for hstore " + storeName + " is " +
|
LOG.debug("maximum sequence id for hstore " + storeName + " is " +
|
||||||
this.maxSeqId);
|
this.maxSeqId);
|
||||||
|
@ -250,16 +250,6 @@ public class HStore implements HConstants {
|
||||||
" -- continuing. Probably DATA LOSS!", e);
|
" -- continuing. Probably DATA LOSS!", e);
|
||||||
}
|
}
|
||||||
|
|
||||||
// By default, we compact if an HStore has more than
|
|
||||||
// MIN_COMMITS_FOR_COMPACTION map files
|
|
||||||
this.compactionThreshold =
|
|
||||||
conf.getInt("hbase.hstore.compactionThreshold", 3);
|
|
||||||
|
|
||||||
// We used to compact in here before bringing the store online. Instead
|
|
||||||
// get it online quick even if it needs compactions so we can start
|
|
||||||
// taking updates as soon as possible (Once online, can take updates even
|
|
||||||
// during a compaction).
|
|
||||||
|
|
||||||
// Move maxSeqId on by one. Why here? And not in HRegion?
|
// Move maxSeqId on by one. Why here? And not in HRegion?
|
||||||
this.maxSeqId += 1;
|
this.maxSeqId += 1;
|
||||||
|
|
||||||
|
@ -281,23 +271,8 @@ public class HStore implements HConstants {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
HColumnDescriptor getFamily() {
|
||||||
* @param hstoreFiles
|
return this.family;
|
||||||
* @return Maximum sequence number found or -1.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private long getMaxSequenceId(final List<HStoreFile> hstoreFiles)
|
|
||||||
throws IOException {
|
|
||||||
long maxSeqID = -1;
|
|
||||||
for (HStoreFile hsf : hstoreFiles) {
|
|
||||||
long seqid = hsf.loadInfo(fs);
|
|
||||||
if (seqid > 0) {
|
|
||||||
if (seqid > maxSeqID) {
|
|
||||||
maxSeqID = seqid;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return maxSeqID;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
long getMaxSequenceId() {
|
long getMaxSequenceId() {
|
||||||
|
@ -388,7 +363,7 @@ public class HStore implements HConstants {
|
||||||
* @param mapdir qualified path for map file directory
|
* @param mapdir qualified path for map file directory
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
private List<HStoreFile> loadHStoreFiles(Path infodir, Path mapdir)
|
private SortedMap<Long, HStoreFile> loadHStoreFiles(Path infodir, Path mapdir)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("infodir: " + infodir.toString() + " mapdir: " +
|
LOG.debug("infodir: " + infodir.toString() + " mapdir: " +
|
||||||
|
@ -397,7 +372,7 @@ public class HStore implements HConstants {
|
||||||
// Look first at info files. If a reference, these contain info we need
|
// Look first at info files. If a reference, these contain info we need
|
||||||
// to create the HStoreFile.
|
// to create the HStoreFile.
|
||||||
FileStatus infofiles[] = fs.listStatus(infodir);
|
FileStatus infofiles[] = fs.listStatus(infodir);
|
||||||
ArrayList<HStoreFile> results = new ArrayList<HStoreFile>(infofiles.length);
|
SortedMap<Long, HStoreFile> results = new TreeMap<Long, HStoreFile>();
|
||||||
ArrayList<Path> mapfiles = new ArrayList<Path>(infofiles.length);
|
ArrayList<Path> mapfiles = new ArrayList<Path>(infofiles.length);
|
||||||
for (int i = 0; i < infofiles.length; i++) {
|
for (int i = 0; i < infofiles.length; i++) {
|
||||||
Path p = infofiles[i].getPath();
|
Path p = infofiles[i].getPath();
|
||||||
|
@ -414,6 +389,11 @@ public class HStore implements HConstants {
|
||||||
boolean isReference = isReference(p, m);
|
boolean isReference = isReference(p, m);
|
||||||
long fid = Long.parseLong(m.group(1));
|
long fid = Long.parseLong(m.group(1));
|
||||||
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("loading file " + p.toString() + ", isReference=" +
|
||||||
|
isReference + ", file id=" + fid);
|
||||||
|
}
|
||||||
|
|
||||||
HStoreFile curfile = null;
|
HStoreFile curfile = null;
|
||||||
HStoreFile.Reference reference = null;
|
HStoreFile.Reference reference = null;
|
||||||
if (isReference) {
|
if (isReference) {
|
||||||
|
@ -421,6 +401,22 @@ public class HStore implements HConstants {
|
||||||
}
|
}
|
||||||
curfile = new HStoreFile(conf, fs, basedir, info.getEncodedName(),
|
curfile = new HStoreFile(conf, fs, basedir, info.getEncodedName(),
|
||||||
family.getFamilyName(), fid, reference);
|
family.getFamilyName(), fid, reference);
|
||||||
|
|
||||||
|
storeSize += curfile.length();
|
||||||
|
long storeSeqId = -1;
|
||||||
|
try {
|
||||||
|
storeSeqId = curfile.loadInfo(fs);
|
||||||
|
if (storeSeqId > this.maxSeqId) {
|
||||||
|
this.maxSeqId = storeSeqId;
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
// If the HSTORE_LOGINFOFILE doesn't contain a number, just ignore it.
|
||||||
|
// That means it was built prior to the previous run of HStore, and so
|
||||||
|
// it cannot contain any updates also contained in the log.
|
||||||
|
LOG.info("HSTORE_LOGINFOFILE " + curfile +
|
||||||
|
" does not contain a sequence number - ignoring");
|
||||||
|
}
|
||||||
|
|
||||||
Path mapfile = curfile.getMapFilePath();
|
Path mapfile = curfile.getMapFilePath();
|
||||||
if (!fs.exists(mapfile)) {
|
if (!fs.exists(mapfile)) {
|
||||||
fs.delete(curfile.getInfoFilePath());
|
fs.delete(curfile.getInfoFilePath());
|
||||||
|
@ -432,7 +428,7 @@ public class HStore implements HConstants {
|
||||||
// TODO: Confirm referent exists.
|
// TODO: Confirm referent exists.
|
||||||
|
|
||||||
// Found map and sympathetic info file. Add this hstorefile to result.
|
// Found map and sympathetic info file. Add this hstorefile to result.
|
||||||
results.add(curfile);
|
results.put(storeSeqId, curfile);
|
||||||
// Keep list of sympathetic data mapfiles for cleaning info dir in next
|
// Keep list of sympathetic data mapfiles for cleaning info dir in next
|
||||||
// section. Make sure path is fully qualified for compare.
|
// section. Make sure path is fully qualified for compare.
|
||||||
mapfiles.add(mapfile);
|
mapfiles.add(mapfile);
|
||||||
|
@ -581,9 +577,7 @@ public class HStore implements HConstants {
|
||||||
for (MapFile.Reader reader: this.readers.values()) {
|
for (MapFile.Reader reader: this.readers.values()) {
|
||||||
reader.close();
|
reader.close();
|
||||||
}
|
}
|
||||||
this.readers.clear();
|
|
||||||
result = new ArrayList<HStoreFile>(storefiles.values());
|
result = new ArrayList<HStoreFile>(storefiles.values());
|
||||||
this.storefiles.clear();
|
|
||||||
LOG.debug("closed " + this.storeName);
|
LOG.debug("closed " + this.storeName);
|
||||||
return result;
|
return result;
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -591,7 +585,6 @@ public class HStore implements HConstants {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
// Flush changes to disk
|
// Flush changes to disk
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
|
@ -644,18 +637,23 @@ public class HStore implements HConstants {
|
||||||
// Related, looks like 'merging compactions' in BigTable paper interlaces
|
// Related, looks like 'merging compactions' in BigTable paper interlaces
|
||||||
// a memcache flush. We don't.
|
// a memcache flush. We don't.
|
||||||
int entries = 0;
|
int entries = 0;
|
||||||
|
long cacheSize = 0;
|
||||||
try {
|
try {
|
||||||
for (Map.Entry<HStoreKey, byte []> es: cache.entrySet()) {
|
for (Map.Entry<HStoreKey, byte []> es: cache.entrySet()) {
|
||||||
HStoreKey curkey = es.getKey();
|
HStoreKey curkey = es.getKey();
|
||||||
|
byte[] bytes = es.getValue();
|
||||||
TextSequence f = HStoreKey.extractFamily(curkey.getColumn());
|
TextSequence f = HStoreKey.extractFamily(curkey.getColumn());
|
||||||
if (f.equals(this.family.getFamilyName())) {
|
if (f.equals(this.family.getFamilyName())) {
|
||||||
entries++;
|
entries++;
|
||||||
out.append(curkey, new ImmutableBytesWritable(es.getValue()));
|
out.append(curkey, new ImmutableBytesWritable(bytes));
|
||||||
|
cacheSize += curkey.getSize() + (bytes != null ? bytes.length : 0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
out.close();
|
out.close();
|
||||||
}
|
}
|
||||||
|
long newStoreSize = flushedFile.length();
|
||||||
|
storeSize += newStoreSize;
|
||||||
|
|
||||||
// B. Write out the log sequence number that corresponds to this output
|
// B. Write out the log sequence number that corresponds to this output
|
||||||
// MapFile. The MapFile is current up to and including the log seq num.
|
// MapFile. The MapFile is current up to and including the log seq num.
|
||||||
|
@ -676,14 +674,14 @@ public class HStore implements HConstants {
|
||||||
this.storefiles.put(flushid, flushedFile);
|
this.storefiles.put(flushid, flushedFile);
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Added " + name + " with " + entries +
|
LOG.debug("Added " + name + " with " + entries +
|
||||||
" entries, sequence id " + logCacheFlushId + ", and size " +
|
" entries, sequence id " + logCacheFlushId + ", data size " +
|
||||||
StringUtils.humanReadableInt(flushedFile.length()) + " for " +
|
StringUtils.humanReadableInt(cacheSize) + ", file size " +
|
||||||
|
StringUtils.humanReadableInt(newStoreSize) + " for " +
|
||||||
this.storeName);
|
this.storeName);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
this.lock.writeLock().unlock();
|
this.lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -691,28 +689,6 @@ public class HStore implements HConstants {
|
||||||
// Compaction
|
// Compaction
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
|
|
||||||
/**
|
|
||||||
* @return True if this store needs compaction.
|
|
||||||
*/
|
|
||||||
boolean needsCompaction() {
|
|
||||||
return this.storefiles != null &&
|
|
||||||
(this.storefiles.size() >= this.compactionThreshold || hasReferences());
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* @return True if this store has references.
|
|
||||||
*/
|
|
||||||
private boolean hasReferences() {
|
|
||||||
if (this.storefiles != null) {
|
|
||||||
for (HStoreFile hsf: this.storefiles.values()) {
|
|
||||||
if (hsf.isReference()) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compact the back-HStores. This method may take some time, so the calling
|
* Compact the back-HStores. This method may take some time, so the calling
|
||||||
* thread must be able to block for long periods.
|
* thread must be able to block for long periods.
|
||||||
|
@ -728,34 +704,58 @@ public class HStore implements HConstants {
|
||||||
*
|
*
|
||||||
* We don't want to hold the structureLock for the whole time, as a compact()
|
* We don't want to hold the structureLock for the whole time, as a compact()
|
||||||
* can be lengthy and we want to allow cache-flushes during this period.
|
* can be lengthy and we want to allow cache-flushes during this period.
|
||||||
* @throws IOException
|
|
||||||
*
|
*
|
||||||
* @return true if compaction completed successfully
|
* @return mid key if a split is needed, null otherwise
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
boolean compact() throws IOException {
|
Text compact() throws IOException {
|
||||||
synchronized (compactLock) {
|
synchronized (compactLock) {
|
||||||
|
long maxId = -1;
|
||||||
|
List<HStoreFile> filesToCompact = null;
|
||||||
|
synchronized (storefiles) {
|
||||||
|
filesToCompact = new ArrayList<HStoreFile>(this.storefiles.values());
|
||||||
|
if (filesToCompact.size() < 1) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("started compaction of " + storefiles.size() +
|
LOG.debug("Not compacting " + this.storeName +
|
||||||
|
" because no store files to compact.");
|
||||||
|
}
|
||||||
|
return checkSplit();
|
||||||
|
} else if (filesToCompact.size() == 1) {
|
||||||
|
if (!filesToCompact.get(0).isReference()) {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Not compacting " + this.storeName +
|
||||||
|
" because only one store file and it is not a reference");
|
||||||
|
}
|
||||||
|
return checkSplit();
|
||||||
|
}
|
||||||
|
} else if (filesToCompact.size() < compactionThreshold) {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Not compacting " + this.storeName +
|
||||||
|
" because number of stores " + filesToCompact.size() +
|
||||||
|
" < compaction threshold " + compactionThreshold);
|
||||||
|
}
|
||||||
|
return checkSplit();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!fs.exists(compactionDir) && !fs.mkdirs(compactionDir)) {
|
||||||
|
LOG.warn("Mkdir on " + compactionDir.toString() + " failed");
|
||||||
|
return checkSplit();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("started compaction of " + filesToCompact.size() +
|
||||||
" files using " + compactionDir.toString() + " for " +
|
" files using " + compactionDir.toString() + " for " +
|
||||||
this.storeName);
|
this.storeName);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Storefiles are keyed by sequence id. The oldest file comes first.
|
// Storefiles are keyed by sequence id. The oldest file comes first.
|
||||||
// We need to return out of here a List that has the newest file first.
|
// We need to return out of here a List that has the newest file first.
|
||||||
List<HStoreFile> filesToCompact =
|
|
||||||
new ArrayList<HStoreFile>(this.storefiles.values());
|
|
||||||
Collections.reverse(filesToCompact);
|
Collections.reverse(filesToCompact);
|
||||||
if (filesToCompact.size() < 1 ||
|
|
||||||
(filesToCompact.size() == 1 && !filesToCompact.get(0).isReference())) {
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("nothing to compact for " + this.storeName);
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!fs.exists(compactionDir) && !fs.mkdirs(compactionDir)) {
|
// The max-sequenceID in any of the to-be-compacted TreeMaps is the
|
||||||
LOG.warn("Mkdir on " + compactionDir.toString() + " failed");
|
// last key of storefiles.
|
||||||
return false;
|
|
||||||
|
maxId = this.storefiles.lastKey();
|
||||||
}
|
}
|
||||||
|
|
||||||
// Step through them, writing to the brand-new MapFile
|
// Step through them, writing to the brand-new MapFile
|
||||||
|
@ -771,15 +771,18 @@ public class HStore implements HConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap.
|
// Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap.
|
||||||
// Compute max-sequenceID seen in any of the to-be-compacted TreeMaps.
|
|
||||||
long maxId = getMaxSequenceId(filesToCompact);
|
|
||||||
compactedOutputFile.writeInfo(fs, maxId);
|
compactedOutputFile.writeInfo(fs, maxId);
|
||||||
|
|
||||||
// Move the compaction into place.
|
// Move the compaction into place.
|
||||||
completeCompaction(filesToCompact, compactedOutputFile);
|
completeCompaction(filesToCompact, compactedOutputFile);
|
||||||
return true;
|
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Completed compaction of " + this.storeName +
|
||||||
|
" store size is " + StringUtils.humanReadableInt(storeSize));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return checkSplit();
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Compact passed <code>toCompactFiles</code> into <code>compactedOut</code>.
|
* Compact passed <code>toCompactFiles</code> into <code>compactedOut</code>.
|
||||||
|
@ -975,11 +978,11 @@ public class HStore implements HConstants {
|
||||||
* <pre>
|
* <pre>
|
||||||
* 1) Wait for active scanners to exit
|
* 1) Wait for active scanners to exit
|
||||||
* 2) Acquiring the write-lock
|
* 2) Acquiring the write-lock
|
||||||
* 3) Figuring out what MapFiles are going to be replaced
|
* 3) Moving the new compacted MapFile into place
|
||||||
* 4) Moving the new compacted MapFile into place
|
* 4) Unloading all the replaced MapFiles and close.
|
||||||
* 5) Unloading all the replaced MapFiles.
|
* 5) Deleting all the replaced MapFile files.
|
||||||
* 6) Deleting all the old MapFile files.
|
* 6) Loading the new TreeMap.
|
||||||
* 7) Loading the new TreeMap.
|
* 7) Compute new store size
|
||||||
* 8) Releasing the write-lock
|
* 8) Releasing the write-lock
|
||||||
* 9) Allow new scanners to proceed.
|
* 9) Allow new scanners to proceed.
|
||||||
* </pre>
|
* </pre>
|
||||||
|
@ -1027,6 +1030,7 @@ public class HStore implements HConstants {
|
||||||
|
|
||||||
// 4. and 5. Unload all the replaced MapFiles, close and delete.
|
// 4. and 5. Unload all the replaced MapFiles, close and delete.
|
||||||
|
|
||||||
|
synchronized (storefiles) {
|
||||||
List<Long> toDelete = new ArrayList<Long>();
|
List<Long> toDelete = new ArrayList<Long>();
|
||||||
for (Map.Entry<Long, HStoreFile> e: this.storefiles.entrySet()) {
|
for (Map.Entry<Long, HStoreFile> e: this.storefiles.entrySet()) {
|
||||||
if (!compactedFiles.contains(e.getValue())) {
|
if (!compactedFiles.contains(e.getValue())) {
|
||||||
|
@ -1061,12 +1065,18 @@ public class HStore implements HConstants {
|
||||||
". Files replaced are " + compactedFiles.toString() +
|
". Files replaced are " + compactedFiles.toString() +
|
||||||
" some of which may have been already removed", e);
|
" some of which may have been already removed", e);
|
||||||
}
|
}
|
||||||
|
// 7. Compute new store size
|
||||||
|
storeSize = 0L;
|
||||||
|
for (HStoreFile hsf: storefiles.values()) {
|
||||||
|
storeSize += hsf.length();
|
||||||
|
}
|
||||||
|
}
|
||||||
} finally {
|
} finally {
|
||||||
// 7. Releasing the write-lock
|
// 8. Releasing the write-lock
|
||||||
this.lock.writeLock().unlock();
|
this.lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
// 8. Allow new scanners to proceed.
|
// 9. Allow new scanners to proceed.
|
||||||
newScannerLock.writeLock().unlock();
|
newScannerLock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1572,32 +1582,29 @@ public class HStore implements HConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets size for the store.
|
* Determines if HStore can be split
|
||||||
*
|
*
|
||||||
* @param midKey Gets set to the middle key of the largest splitable store
|
* @return midKey if store can be split, null otherwise
|
||||||
* file or its set to empty if largest is not splitable.
|
|
||||||
* @return Sizes for the store and the passed <code>midKey</code> is
|
|
||||||
* set to midKey of largest splitable. Otherwise, its set to empty
|
|
||||||
* to indicate we couldn't find a midkey to split on
|
|
||||||
*/
|
*/
|
||||||
HStoreSize size(Text midKey) {
|
Text checkSplit() {
|
||||||
long maxSize = 0L;
|
|
||||||
long aggregateSize = 0L;
|
|
||||||
// Not splitable if we find a reference store file present in the store.
|
|
||||||
boolean splitable = true;
|
|
||||||
if (this.storefiles.size() <= 0) {
|
if (this.storefiles.size() <= 0) {
|
||||||
return new HStoreSize(0, 0, splitable);
|
return null;
|
||||||
|
}
|
||||||
|
if (storeSize < this.desiredMaxFileSize) {
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
this.lock.readLock().lock();
|
this.lock.readLock().lock();
|
||||||
try {
|
try {
|
||||||
|
// Not splitable if we find a reference store file present in the store.
|
||||||
|
boolean splitable = true;
|
||||||
|
long maxSize = 0L;
|
||||||
Long mapIndex = Long.valueOf(0L);
|
Long mapIndex = Long.valueOf(0L);
|
||||||
// Iterate through all the MapFiles
|
// Iterate through all the MapFiles
|
||||||
|
synchronized (storefiles) {
|
||||||
for (Map.Entry<Long, HStoreFile> e: storefiles.entrySet()) {
|
for (Map.Entry<Long, HStoreFile> e: storefiles.entrySet()) {
|
||||||
HStoreFile curHSF = e.getValue();
|
HStoreFile curHSF = e.getValue();
|
||||||
long size = curHSF.length();
|
long size = curHSF.length();
|
||||||
aggregateSize += size;
|
if (size > maxSize) {
|
||||||
if (maxSize == 0L || size > maxSize) {
|
|
||||||
// This is the largest one so far
|
// This is the largest one so far
|
||||||
maxSize = size;
|
maxSize = size;
|
||||||
mapIndex = e.getKey();
|
mapIndex = e.getKey();
|
||||||
|
@ -1606,16 +1613,22 @@ public class HStore implements HConstants {
|
||||||
splitable = !curHSF.isReference();
|
splitable = !curHSF.isReference();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (splitable) {
|
}
|
||||||
|
if (!splitable) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
MapFile.Reader r = this.readers.get(mapIndex);
|
MapFile.Reader r = this.readers.get(mapIndex);
|
||||||
|
|
||||||
// seek back to the beginning of mapfile
|
// seek back to the beginning of mapfile
|
||||||
r.reset();
|
r.reset();
|
||||||
|
|
||||||
// get the first and last keys
|
// get the first and last keys
|
||||||
HStoreKey firstKey = new HStoreKey();
|
HStoreKey firstKey = new HStoreKey();
|
||||||
HStoreKey lastKey = new HStoreKey();
|
HStoreKey lastKey = new HStoreKey();
|
||||||
Writable value = new ImmutableBytesWritable();
|
Writable value = new ImmutableBytesWritable();
|
||||||
r.next(firstKey, value);
|
r.next(firstKey, value);
|
||||||
r.finalKey(lastKey);
|
r.finalKey(lastKey);
|
||||||
|
|
||||||
// get the midkey
|
// get the midkey
|
||||||
HStoreKey mk = (HStoreKey)r.midKey();
|
HStoreKey mk = (HStoreKey)r.midKey();
|
||||||
if (mk != null) {
|
if (mk != null) {
|
||||||
|
@ -1623,18 +1636,21 @@ public class HStore implements HConstants {
|
||||||
// (ever) split this region.
|
// (ever) split this region.
|
||||||
if (mk.getRow().equals(firstKey.getRow()) &&
|
if (mk.getRow().equals(firstKey.getRow()) &&
|
||||||
mk.getRow().equals(lastKey.getRow())) {
|
mk.getRow().equals(lastKey.getRow())) {
|
||||||
return new HStoreSize(aggregateSize, maxSize, false);
|
return null;
|
||||||
}
|
|
||||||
// Otherwise, set midKey
|
|
||||||
midKey.set(mk.getRow());
|
|
||||||
}
|
}
|
||||||
|
return mk.getRow();
|
||||||
}
|
}
|
||||||
} catch(IOException e) {
|
} catch(IOException e) {
|
||||||
LOG.warn("Failed getting store size for " + this.storeName, e);
|
LOG.warn("Failed getting store size for " + this.storeName, e);
|
||||||
} finally {
|
} finally {
|
||||||
this.lock.readLock().unlock();
|
this.lock.readLock().unlock();
|
||||||
}
|
}
|
||||||
return new HStoreSize(aggregateSize, maxSize, splitable);
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return aggregate size of HStore */
|
||||||
|
public long getSize() {
|
||||||
|
return storeSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
|
@ -1665,7 +1681,7 @@ public class HStore implements HConstants {
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return this.storeName;
|
return this.storeName.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -1,33 +0,0 @@
|
||||||
|
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Data structure to hold result of a look at store file sizes.
|
|
||||||
*/
|
|
||||||
public class HStoreSize {
|
|
||||||
final long aggregate;
|
|
||||||
final long largest;
|
|
||||||
boolean splitable;
|
|
||||||
|
|
||||||
HStoreSize(final long a, final long l, final boolean s) {
|
|
||||||
this.aggregate = a;
|
|
||||||
this.largest = l;
|
|
||||||
this.splitable = s;
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getAggregate() {
|
|
||||||
return this.aggregate;
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getLargest() {
|
|
||||||
return this.largest;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isSplitable() {
|
|
||||||
return this.splitable;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setSplitable(final boolean s) {
|
|
||||||
this.splitable = s;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -28,7 +28,6 @@ import java.util.Map;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
|
||||||
|
@ -196,12 +195,9 @@ class Memcache {
|
||||||
/**
|
/**
|
||||||
* @param row
|
* @param row
|
||||||
* @param timestamp
|
* @param timestamp
|
||||||
* @return the key that matches <i>row</i> exactly, or the one that
|
|
||||||
* immediately preceeds it.
|
|
||||||
*/
|
*/
|
||||||
void getRowKeyAtOrBefore(final Text row,
|
void getRowKeyAtOrBefore(final Text row,
|
||||||
SortedMap<HStoreKey, Long> candidateKeys)
|
SortedMap<HStoreKey, Long> candidateKeys) {
|
||||||
throws IOException {
|
|
||||||
this.lock.readLock().lock();
|
this.lock.readLock().lock();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -1,78 +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.regionserver;
|
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import java.util.concurrent.Delayed;
|
|
||||||
|
|
||||||
/** Queue entry passed to flusher, compactor and splitter threads */
|
|
||||||
class QueueEntry implements Delayed {
|
|
||||||
private final HRegion region;
|
|
||||||
private long expirationTime;
|
|
||||||
|
|
||||||
QueueEntry(HRegion region, long expirationTime) {
|
|
||||||
this.region = region;
|
|
||||||
this.expirationTime = expirationTime;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
QueueEntry other = (QueueEntry) o;
|
|
||||||
return this.hashCode() == other.hashCode();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return this.region.getRegionInfo().hashCode();
|
|
||||||
}
|
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public long getDelay(TimeUnit unit) {
|
|
||||||
return unit.convert(this.expirationTime - System.currentTimeMillis(),
|
|
||||||
TimeUnit.MILLISECONDS);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public int compareTo(Delayed o) {
|
|
||||||
long delta = this.getDelay(TimeUnit.MILLISECONDS) -
|
|
||||||
o.getDelay(TimeUnit.MILLISECONDS);
|
|
||||||
|
|
||||||
int value = 0;
|
|
||||||
if (delta > 0) {
|
|
||||||
value = 1;
|
|
||||||
|
|
||||||
} else if (delta < 0) {
|
|
||||||
value = -1;
|
|
||||||
}
|
|
||||||
return value;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** @return the region */
|
|
||||||
public HRegion getRegion() {
|
|
||||||
return region;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** @param expirationTime the expirationTime to set */
|
|
||||||
public void setExpirationTime(long expirationTime) {
|
|
||||||
this.expirationTime = expirationTime;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -35,12 +35,13 @@ import org.apache.hadoop.io.Text;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
|
import org.apache.hadoop.hbase.io.Cell;
|
||||||
import org.apache.hadoop.hbase.regionserver.HLog;
|
import org.apache.hadoop.hbase.regionserver.HLog;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HScannerInterface;
|
import org.apache.hadoop.hbase.HScannerInterface;
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
import org.apache.hadoop.hbase.io.Cell;
|
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -316,13 +317,16 @@ public class MetaUtils {
|
||||||
throws IOException {
|
throws IOException {
|
||||||
HTable t = new HTable(c, HConstants.META_TABLE_NAME);
|
HTable t = new HTable(c, HConstants.META_TABLE_NAME);
|
||||||
Cell cell = t.get(row, HConstants.COL_REGIONINFO);
|
Cell cell = t.get(row, HConstants.COL_REGIONINFO);
|
||||||
|
if (cell == null) {
|
||||||
|
throw new IOException("no information for row " + row);
|
||||||
|
}
|
||||||
// Throws exception if null.
|
// Throws exception if null.
|
||||||
HRegionInfo info = Writables.getHRegionInfo(cell);
|
HRegionInfo info = Writables.getHRegionInfo(cell);
|
||||||
long id = t.startUpdate(row);
|
BatchUpdate b = new BatchUpdate(row);
|
||||||
info.setOffline(onlineOffline);
|
info.setOffline(onlineOffline);
|
||||||
t.put(id, HConstants.COL_REGIONINFO, Writables.getBytes(info));
|
b.put(HConstants.COL_REGIONINFO, Writables.getBytes(info));
|
||||||
t.delete(id, HConstants.COL_SERVER);
|
b.delete(HConstants.COL_SERVER);
|
||||||
t.delete(id, HConstants.COL_STARTCODE);
|
b.delete(HConstants.COL_STARTCODE);
|
||||||
t.commit(id);
|
t.commit(b);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,6 +45,7 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||||
protected int regionServers;
|
protected int regionServers;
|
||||||
protected boolean startDfs;
|
protected boolean startDfs;
|
||||||
|
|
||||||
|
/** default constructor */
|
||||||
public HBaseClusterTestCase() {
|
public HBaseClusterTestCase() {
|
||||||
this(1);
|
this(1);
|
||||||
}
|
}
|
||||||
|
@ -53,6 +54,7 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||||
* Start a MiniHBaseCluster with regionServers region servers in-process to
|
* Start a MiniHBaseCluster with regionServers region servers in-process to
|
||||||
* start with. Also, start a MiniDfsCluster before starting the hbase cluster.
|
* start with. Also, start a MiniDfsCluster before starting the hbase cluster.
|
||||||
* The configuration used will be edited so that this works correctly.
|
* The configuration used will be edited so that this works correctly.
|
||||||
|
* @param regionServers number of region servers to start.
|
||||||
*/
|
*/
|
||||||
public HBaseClusterTestCase(int regionServers) {
|
public HBaseClusterTestCase(int regionServers) {
|
||||||
this(regionServers, true);
|
this(regionServers, true);
|
||||||
|
@ -65,6 +67,8 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||||
* configured in hbase-site.xml and is already started, or you have started a
|
* 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
|
* MiniDFSCluster on your own and edited the configuration in memory. (You
|
||||||
* can modify the config used by overriding the preHBaseClusterSetup method.)
|
* 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) {
|
public HBaseClusterTestCase(int regionServers, boolean startDfs) {
|
||||||
super();
|
super();
|
||||||
|
@ -81,9 +85,11 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||||
/**
|
/**
|
||||||
* Actually start the MiniHBase instance.
|
* Actually start the MiniHBase instance.
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
protected void HBaseClusterSetup() throws Exception {
|
protected void HBaseClusterSetup() throws Exception {
|
||||||
// start the mini cluster
|
// start the mini cluster
|
||||||
this.cluster = new MiniHBaseCluster(conf, regionServers);
|
this.cluster = new MiniHBaseCluster(conf, regionServers);
|
||||||
|
// opening the META table ensures that cluster is running
|
||||||
HTable meta = new HTable(conf, new Text(".META."));
|
HTable meta = new HTable(conf, new Text(".META."));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -102,11 +102,8 @@ public abstract class HBaseTestCase extends TestCase {
|
||||||
localfs =
|
localfs =
|
||||||
(conf.get("fs.default.name", "file:///").compareTo("file::///") == 0);
|
(conf.get("fs.default.name", "file:///").compareTo("file::///") == 0);
|
||||||
|
|
||||||
try {
|
if (fs == null) {
|
||||||
this.fs = FileSystem.get(conf);
|
this.fs = FileSystem.get(conf);
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.fatal("error getting file system", e);
|
|
||||||
throw e;
|
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
if (localfs) {
|
if (localfs) {
|
||||||
|
@ -509,54 +506,88 @@ public abstract class HBaseTestCase extends TestCase {
|
||||||
*/
|
*/
|
||||||
public static class HTableIncommon implements Incommon {
|
public static class HTableIncommon implements Incommon {
|
||||||
final HTable table;
|
final HTable table;
|
||||||
|
private BatchUpdate batch;
|
||||||
|
|
||||||
|
private void checkBatch() {
|
||||||
|
if (batch == null) {
|
||||||
|
throw new IllegalStateException("No batch update in progress.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param table
|
* @param table
|
||||||
*/
|
*/
|
||||||
public HTableIncommon(final HTable table) {
|
public HTableIncommon(final HTable table) {
|
||||||
super();
|
super();
|
||||||
this.table = table;
|
this.table = table;
|
||||||
|
this.batch = null;
|
||||||
}
|
}
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void abort(long lockid) {
|
public void abort(@SuppressWarnings("unused") long lockid) {
|
||||||
this.table.abort(lockid);
|
if (this.batch != null) {
|
||||||
|
this.batch = null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void commit(long lockid) throws IOException {
|
public void commit(@SuppressWarnings("unused") long lockid)
|
||||||
this.table.commit(lockid);
|
throws IOException {
|
||||||
|
checkBatch();
|
||||||
|
this.table.commit(batch);
|
||||||
|
this.batch = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void commit(long lockid, final long ts) throws IOException {
|
public void commit(@SuppressWarnings("unused") long lockid, final long ts)
|
||||||
this.table.commit(lockid, ts);
|
throws IOException {
|
||||||
|
checkBatch();
|
||||||
|
this.batch.setTimestamp(ts);
|
||||||
|
this.table.commit(batch);
|
||||||
|
this.batch = null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void put(long lockid, Text column, byte[] val) {
|
public void put(@SuppressWarnings("unused") long lockid, Text column,
|
||||||
this.table.put(lockid, column, val);
|
byte[] val) {
|
||||||
|
checkBatch();
|
||||||
|
this.batch.put(column, val);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void delete(long lockid, Text column) {
|
public void delete(@SuppressWarnings("unused") long lockid, Text column) {
|
||||||
this.table.delete(lockid, column);
|
checkBatch();
|
||||||
|
this.batch.delete(column);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void deleteAll(Text row, Text column, long ts) throws IOException {
|
public void deleteAll(Text row, Text column, long ts) throws IOException {
|
||||||
this.table.deleteAll(row, column, ts);
|
this.table.deleteAll(row, column, ts);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public long startUpdate(Text row) {
|
public long startUpdate(Text row) {
|
||||||
return this.table.startUpdate(row);
|
if (this.batch != null) {
|
||||||
|
throw new IllegalStateException("Batch update already in progress.");
|
||||||
}
|
}
|
||||||
|
this.batch = new BatchUpdate(row);
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public HScannerInterface getScanner(Text [] columns, Text firstRow,
|
public HScannerInterface getScanner(Text [] columns, Text firstRow,
|
||||||
long ts) throws IOException {
|
long ts) throws IOException {
|
||||||
return this.table.obtainScanner(columns, firstRow, ts, null);
|
return this.table.obtainScanner(columns, firstRow, ts, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public Cell get(Text row, Text column) throws IOException {
|
public Cell get(Text row, Text column) throws IOException {
|
||||||
return this.table.get(row, column);
|
return this.table.get(row, column);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public Cell[] get(Text row, Text column, int versions) throws IOException {
|
public Cell[] get(Text row, Text column, int versions) throws IOException {
|
||||||
return this.table.get(row, column, versions);
|
return this.table.get(row, column, versions);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public Cell[] get(Text row, Text column, long ts, int versions)
|
public Cell[] get(Text row, Text column, long ts, int versions)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -576,8 +607,10 @@ public abstract class HBaseTestCase extends TestCase {
|
||||||
fail(column.toString() + " at timestamp " + timestamp +
|
fail(column.toString() + " at timestamp " + timestamp +
|
||||||
"\" was expected to be \"" + value + " but was null");
|
"\" was expected to be \"" + value + " but was null");
|
||||||
}
|
}
|
||||||
|
if (cell_value != null) {
|
||||||
assertEquals(column.toString() + " at timestamp "
|
assertEquals(column.toString() + " at timestamp "
|
||||||
+ timestamp, value, new String(cell_value.getValue()));
|
+ timestamp, value, new String(cell_value.getValue()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -24,6 +24,8 @@ import java.util.Iterator;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
|
@ -33,6 +35,7 @@ import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
* Test HBase Master and Region servers, client API
|
* Test HBase Master and Region servers, client API
|
||||||
*/
|
*/
|
||||||
public class TestHBaseCluster extends HBaseClusterTestCase {
|
public class TestHBaseCluster extends HBaseClusterTestCase {
|
||||||
|
private static final Log LOG = LogFactory.getLog(TestHBaseCluster.class);
|
||||||
|
|
||||||
private HTableDescriptor desc;
|
private HTableDescriptor desc;
|
||||||
private HBaseAdmin admin;
|
private HBaseAdmin admin;
|
||||||
|
@ -104,7 +107,7 @@ public class TestHBaseCluster extends HBaseClusterTestCase {
|
||||||
(ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
|
(ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
|
||||||
table.commit(b);
|
table.commit(b);
|
||||||
}
|
}
|
||||||
System.out.println("Write " + NUM_VALS + " rows. Elapsed time: "
|
LOG.info("Write " + NUM_VALS + " rows. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// Read them back in
|
// Read them back in
|
||||||
|
@ -134,7 +137,7 @@ public class TestHBaseCluster extends HBaseClusterTestCase {
|
||||||
teststr.compareTo(bodystr) == 0);
|
teststr.compareTo(bodystr) == 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
System.out.println("Read " + NUM_VALS + " rows. Elapsed time: "
|
LOG.info("Read " + NUM_VALS + " rows. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -175,7 +178,7 @@ public class TestHBaseCluster extends HBaseClusterTestCase {
|
||||||
anchorFetched++;
|
anchorFetched++;
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
System.out.println(col);
|
LOG.info(col);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
curVals.clear();
|
curVals.clear();
|
||||||
|
@ -184,7 +187,7 @@ public class TestHBaseCluster extends HBaseClusterTestCase {
|
||||||
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
|
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
|
||||||
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
|
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + NUM_VALS
|
LOG.info("Scanned " + NUM_VALS
|
||||||
+ " rows. Elapsed time: "
|
+ " rows. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.client.HTable;
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.MultiRegionTable;
|
import org.apache.hadoop.hbase.MultiRegionTable;
|
||||||
import org.apache.hadoop.hbase.StaticTestEnvironment;
|
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.mapred.JobClient;
|
import org.apache.hadoop.mapred.JobClient;
|
||||||
import org.apache.hadoop.mapred.JobConf;
|
import org.apache.hadoop.mapred.JobConf;
|
||||||
|
@ -76,9 +76,9 @@ public class TestTableIndex extends MultiRegionTable {
|
||||||
|
|
||||||
|
|
||||||
private HTableDescriptor desc;
|
private HTableDescriptor desc;
|
||||||
|
private JobConf jobConf = null;
|
||||||
|
|
||||||
private Path dir;
|
/** default constructor */
|
||||||
|
|
||||||
public TestTableIndex() {
|
public TestTableIndex() {
|
||||||
// Enable DEBUG-level MR logging.
|
// Enable DEBUG-level MR logging.
|
||||||
Logger.getLogger("org.apache.hadoop.mapred").setLevel(Level.DEBUG);
|
Logger.getLogger("org.apache.hadoop.mapred").setLevel(Level.DEBUG);
|
||||||
|
@ -105,7 +105,6 @@ public class TestTableIndex extends MultiRegionTable {
|
||||||
// Create a table.
|
// Create a table.
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||||
admin.createTable(desc);
|
admin.createTable(desc);
|
||||||
|
|
||||||
// Populate a table into multiple regions
|
// Populate a table into multiple regions
|
||||||
makeMultiRegionTable(conf, cluster, dfsCluster.getFileSystem(), TABLE_NAME,
|
makeMultiRegionTable(conf, cluster, dfsCluster.getFileSystem(), TABLE_NAME,
|
||||||
INPUT_COLUMN);
|
INPUT_COLUMN);
|
||||||
|
@ -116,6 +115,14 @@ public class TestTableIndex extends MultiRegionTable {
|
||||||
assertTrue(startKeys.length > 1);
|
assertTrue(startKeys.length > 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
if (jobConf != null) {
|
||||||
|
FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test HBase map/reduce
|
* Test HBase map/reduce
|
||||||
*
|
*
|
||||||
|
@ -135,7 +142,7 @@ public class TestTableIndex extends MultiRegionTable {
|
||||||
conf.set("hbase.index.conf", createIndexConfContent());
|
conf.set("hbase.index.conf", createIndexConfContent());
|
||||||
|
|
||||||
try {
|
try {
|
||||||
JobConf jobConf = new JobConf(conf, TestTableIndex.class);
|
jobConf = new JobConf(conf, TestTableIndex.class);
|
||||||
jobConf.setJobName("index column contents");
|
jobConf.setJobName("index column contents");
|
||||||
jobConf.setNumMapTasks(2);
|
jobConf.setNumMapTasks(2);
|
||||||
// number of indexes to partition into
|
// number of indexes to partition into
|
||||||
|
|
|
@ -20,13 +20,14 @@
|
||||||
package org.apache.hadoop.hbase.mapred;
|
package org.apache.hadoop.hbase.mapred;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.File;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.client.HTable;
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.MultiRegionTable;
|
import org.apache.hadoop.hbase.MultiRegionTable;
|
||||||
import org.apache.hadoop.hbase.StaticTestEnvironment;
|
|
||||||
import org.apache.hadoop.hbase.io.BatchUpdate;
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
import org.apache.hadoop.io.MapWritable;
|
import org.apache.hadoop.io.MapWritable;
|
||||||
|
@ -66,8 +66,6 @@ public class TestTableMapReduce extends MultiRegionTable {
|
||||||
TEXT_OUTPUT_COLUMN
|
TEXT_OUTPUT_COLUMN
|
||||||
};
|
};
|
||||||
|
|
||||||
private Path dir;
|
|
||||||
|
|
||||||
private static byte[][] values = null;
|
private static byte[][] values = null;
|
||||||
|
|
||||||
static {
|
static {
|
||||||
|
@ -193,8 +191,9 @@ public class TestTableMapReduce extends MultiRegionTable {
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
|
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
|
||||||
|
|
||||||
|
JobConf jobConf = null;
|
||||||
try {
|
try {
|
||||||
JobConf jobConf = new JobConf(conf, TestTableMapReduce.class);
|
jobConf = new JobConf(conf, TestTableMapReduce.class);
|
||||||
jobConf.setJobName("process column contents");
|
jobConf.setJobName("process column contents");
|
||||||
jobConf.setNumMapTasks(1);
|
jobConf.setNumMapTasks(1);
|
||||||
jobConf.setNumReduceTasks(1);
|
jobConf.setNumReduceTasks(1);
|
||||||
|
@ -215,6 +214,9 @@ public class TestTableMapReduce extends MultiRegionTable {
|
||||||
verify(SINGLE_REGION_TABLE_NAME);
|
verify(SINGLE_REGION_TABLE_NAME);
|
||||||
} finally {
|
} finally {
|
||||||
mrCluster.shutdown();
|
mrCluster.shutdown();
|
||||||
|
if (jobConf != null) {
|
||||||
|
FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir")));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -244,8 +246,9 @@ public class TestTableMapReduce extends MultiRegionTable {
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
|
MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
|
||||||
|
|
||||||
|
JobConf jobConf = null;
|
||||||
try {
|
try {
|
||||||
JobConf jobConf = new JobConf(conf, TestTableMapReduce.class);
|
jobConf = new JobConf(conf, TestTableMapReduce.class);
|
||||||
jobConf.setJobName("process column contents");
|
jobConf.setJobName("process column contents");
|
||||||
jobConf.setNumMapTasks(2);
|
jobConf.setNumMapTasks(2);
|
||||||
jobConf.setNumReduceTasks(1);
|
jobConf.setNumReduceTasks(1);
|
||||||
|
@ -262,6 +265,9 @@ public class TestTableMapReduce extends MultiRegionTable {
|
||||||
verify(MULTI_REGION_TABLE_NAME);
|
verify(MULTI_REGION_TABLE_NAME);
|
||||||
} finally {
|
} finally {
|
||||||
mrCluster.shutdown();
|
mrCluster.shutdown();
|
||||||
|
if (jobConf != null) {
|
||||||
|
FileUtil.fullyDelete(new File(jobConf.get("hadoop.tmp.dir")));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -93,7 +93,6 @@ public class TestCompaction extends HBaseTestCase {
|
||||||
*/
|
*/
|
||||||
public void testCompaction() throws Exception {
|
public void testCompaction() throws Exception {
|
||||||
createStoreFile(r);
|
createStoreFile(r);
|
||||||
assertFalse(r.compactIfNeeded());
|
|
||||||
for (int i = 0; i < COMPACTION_THRESHOLD; i++) {
|
for (int i = 0; i < COMPACTION_THRESHOLD; i++) {
|
||||||
createStoreFile(r);
|
createStoreFile(r);
|
||||||
}
|
}
|
||||||
|
@ -106,35 +105,8 @@ public class TestCompaction extends HBaseTestCase {
|
||||||
r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/);
|
r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/);
|
||||||
// Assert that I can get > 5 versions (Should be at least 5 in there).
|
// Assert that I can get > 5 versions (Should be at least 5 in there).
|
||||||
assertTrue(cellValues.length >= 5);
|
assertTrue(cellValues.length >= 5);
|
||||||
// Try to run compaction concurrent with a thread flush just to see that
|
r.flushcache();
|
||||||
// we can.
|
r.compactStores();
|
||||||
final HRegion region = this.r;
|
|
||||||
Thread t1 = new Thread() {
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
try {
|
|
||||||
region.flushcache();
|
|
||||||
} catch (IOException e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
Thread t2 = new Thread() {
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
try {
|
|
||||||
assertTrue(region.compactIfNeeded());
|
|
||||||
} catch (IOException e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
t1.setDaemon(true);
|
|
||||||
t1.start();
|
|
||||||
t2.setDaemon(true);
|
|
||||||
t2.start();
|
|
||||||
t1.join();
|
|
||||||
t2.join();
|
|
||||||
// Now assert that there are 4 versions of a record only: thats the
|
// Now assert that there are 4 versions of a record only: thats the
|
||||||
// 3 versions that should be in the compacted store and then the one more
|
// 3 versions that should be in the compacted store and then the one more
|
||||||
// we added when we flushed. But could be 3 only if the flush happened
|
// we added when we flushed. But could be 3 only if the flush happened
|
||||||
|
@ -170,7 +142,8 @@ public class TestCompaction extends HBaseTestCase {
|
||||||
// compacted store and the flush above when we added deletes. Add more
|
// compacted store and the flush above when we added deletes. Add more
|
||||||
// content to be certain.
|
// content to be certain.
|
||||||
createSmallerStoreFile(this.r);
|
createSmallerStoreFile(this.r);
|
||||||
assertTrue(r.compactIfNeeded());
|
r.flushcache();
|
||||||
|
r.compactStores();
|
||||||
// Assert that the first row is still deleted.
|
// Assert that the first row is still deleted.
|
||||||
cellValues = r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/);
|
cellValues = r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/);
|
||||||
assertNull(cellValues);
|
assertNull(cellValues);
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.SortedMap;
|
import java.util.SortedMap;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
@ -44,6 +45,13 @@ public class TestHMemcache extends TestCase {
|
||||||
|
|
||||||
private static final String COLUMN_FAMILY = "column";
|
private static final String COLUMN_FAMILY = "column";
|
||||||
|
|
||||||
|
private static final int FIRST_ROW = 1;
|
||||||
|
private static final int NUM_VALS = 1000;
|
||||||
|
private static final Text CONTENTS_BASIC = new Text("contents:basic");
|
||||||
|
private static final String CONTENTSTR = "contentstr";
|
||||||
|
private static final String ANCHORNUM = "anchor:anchornum-";
|
||||||
|
private static final String ANCHORSTR = "anchorstr";
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
|
@ -51,6 +59,50 @@ public class TestHMemcache extends TestCase {
|
||||||
this.hmemcache = new Memcache();
|
this.hmemcache = new Memcache();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @throws UnsupportedEncodingException
|
||||||
|
*/
|
||||||
|
public void testMemcache() throws UnsupportedEncodingException {
|
||||||
|
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||||
|
Text row = new Text("row_" + k);
|
||||||
|
HStoreKey key =
|
||||||
|
new HStoreKey(row, CONTENTS_BASIC, System.currentTimeMillis());
|
||||||
|
hmemcache.add(key, (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING));
|
||||||
|
|
||||||
|
key =
|
||||||
|
new HStoreKey(row, new Text(ANCHORNUM + k), System.currentTimeMillis());
|
||||||
|
hmemcache.add(key, (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Read them back
|
||||||
|
|
||||||
|
for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
|
||||||
|
List<Cell> results;
|
||||||
|
Text row = new Text("row_" + k);
|
||||||
|
HStoreKey key = new HStoreKey(row, CONTENTS_BASIC, Long.MAX_VALUE);
|
||||||
|
results = hmemcache.get(key, 1);
|
||||||
|
assertNotNull("no data for " + key.toString(), results);
|
||||||
|
assertEquals(1, results.size());
|
||||||
|
String bodystr = new String(results.get(0).getValue(),
|
||||||
|
HConstants.UTF8_ENCODING);
|
||||||
|
String teststr = CONTENTSTR + k;
|
||||||
|
assertTrue("Incorrect value for key: (" + key.toString() +
|
||||||
|
"), expected: '" + teststr + "' got: '" +
|
||||||
|
bodystr + "'", teststr.compareTo(bodystr) == 0);
|
||||||
|
|
||||||
|
key = new HStoreKey(row, new Text(ANCHORNUM + k), Long.MAX_VALUE);
|
||||||
|
results = hmemcache.get(key, 1);
|
||||||
|
assertNotNull("no data for " + key.toString(), results);
|
||||||
|
assertEquals(1, results.size());
|
||||||
|
bodystr = new String(results.get(0).getValue(),
|
||||||
|
HConstants.UTF8_ENCODING);
|
||||||
|
teststr = ANCHORSTR + k;
|
||||||
|
assertTrue("Incorrect value for key: (" + key.toString() +
|
||||||
|
"), expected: '" + teststr + "' got: '" + bodystr + "'",
|
||||||
|
teststr.compareTo(bodystr) == 0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private Text getRowName(final int index) {
|
private Text getRowName(final int index) {
|
||||||
return new Text("row" + Integer.toString(index));
|
return new Text("row" + Integer.toString(index));
|
||||||
}
|
}
|
||||||
|
@ -175,8 +227,8 @@ public class TestHMemcache extends TestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** For HBASE-528 **/
|
/** For HBASE-528 */
|
||||||
public void testGetRowKeyAtOrBefore() throws IOException {
|
public void testGetRowKeyAtOrBefore() {
|
||||||
// set up some test data
|
// set up some test data
|
||||||
Text t10 = new Text("010");
|
Text t10 = new Text("010");
|
||||||
Text t20 = new Text("020");
|
Text t20 = new Text("020");
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -57,7 +56,7 @@ implements RegionUnavailableListener {
|
||||||
*/
|
*/
|
||||||
public void testHRegion() throws IOException {
|
public void testHRegion() throws IOException {
|
||||||
try {
|
try {
|
||||||
setup();
|
init();
|
||||||
locks();
|
locks();
|
||||||
badPuts();
|
badPuts();
|
||||||
basic();
|
basic();
|
||||||
|
@ -65,14 +64,7 @@ implements RegionUnavailableListener {
|
||||||
batchWrite();
|
batchWrite();
|
||||||
splitAndMerge();
|
splitAndMerge();
|
||||||
read();
|
read();
|
||||||
cleanup();
|
|
||||||
} finally {
|
} finally {
|
||||||
if (r != null) {
|
|
||||||
r.close();
|
|
||||||
}
|
|
||||||
if (log != null) {
|
|
||||||
log.closeAndDelete();
|
|
||||||
}
|
|
||||||
StaticTestEnvironment.shutdownDfs(cluster);
|
StaticTestEnvironment.shutdownDfs(cluster);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -97,21 +89,35 @@ implements RegionUnavailableListener {
|
||||||
|
|
||||||
private static int numInserted = 0;
|
private static int numInserted = 0;
|
||||||
|
|
||||||
// Create directories, start mini cluster, etc.
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
this.conf.set("hbase.hstore.compactionThreshold", "2");
|
||||||
|
|
||||||
private void setup() throws IOException {
|
if (!StaticTestEnvironment.debugging) {
|
||||||
|
conf.setLong("hbase.hregion.max.filesize", 65536);
|
||||||
|
}
|
||||||
|
|
||||||
cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||||
|
fs = cluster.getFileSystem();
|
||||||
|
|
||||||
// Set the hbase.rootdir to be the home directory in mini dfs.
|
// Set the hbase.rootdir to be the home directory in mini dfs.
|
||||||
this.conf.set(HConstants.HBASE_DIR,
|
this.conf.set(HConstants.HBASE_DIR,
|
||||||
this.cluster.getFileSystem().getHomeDirectory().toString());
|
this.cluster.getFileSystem().getHomeDirectory().toString());
|
||||||
|
|
||||||
|
super.setUp();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create directories, start mini cluster, etc.
|
||||||
|
|
||||||
|
private void init() throws IOException {
|
||||||
desc = new HTableDescriptor("test");
|
desc = new HTableDescriptor("test");
|
||||||
desc.addFamily(new HColumnDescriptor("contents:"));
|
desc.addFamily(new HColumnDescriptor("contents:"));
|
||||||
desc.addFamily(new HColumnDescriptor("anchor:"));
|
desc.addFamily(new HColumnDescriptor("anchor:"));
|
||||||
r = createNewHRegion(desc, null, null);
|
r = createNewHRegion(desc, null, null);
|
||||||
log = r.getLog();
|
log = r.getLog();
|
||||||
region = new HRegionIncommon(r);
|
region = new HRegionIncommon(r);
|
||||||
|
LOG.info("setup completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
// Test basic functionality. Writes to contents:basic and anchor:anchornum-*
|
// Test basic functionality. Writes to contents:basic and anchor:anchornum-*
|
||||||
|
@ -129,7 +135,7 @@ implements RegionUnavailableListener {
|
||||||
(ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
|
(ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
|
||||||
region.commit(writeid, System.currentTimeMillis());
|
region.commit(writeid, System.currentTimeMillis());
|
||||||
}
|
}
|
||||||
System.out.println("Write " + NUM_VALS + " rows. Elapsed time: "
|
LOG.info("Write " + NUM_VALS + " rows. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// Flush cache
|
// Flush cache
|
||||||
|
@ -138,7 +144,7 @@ implements RegionUnavailableListener {
|
||||||
|
|
||||||
region.flushcache();
|
region.flushcache();
|
||||||
|
|
||||||
System.out.println("Cache flush elapsed time: "
|
LOG.info("Cache flush elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// Read them back in
|
// Read them back in
|
||||||
|
@ -165,8 +171,10 @@ implements RegionUnavailableListener {
|
||||||
bodystr, teststr);
|
bodystr, teststr);
|
||||||
}
|
}
|
||||||
|
|
||||||
System.out.println("Read " + NUM_VALS + " rows. Elapsed time: "
|
LOG.info("Read " + NUM_VALS + " rows. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
LOG.info("basic completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
private void badPuts() {
|
private void badPuts() {
|
||||||
|
@ -198,6 +206,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
assertTrue("Bad family", exceptionThrown);
|
assertTrue("Bad family", exceptionThrown);
|
||||||
|
LOG.info("badPuts completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -253,6 +262,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
LOG.info("locks completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
// Test scanners. Writes contents:firstcol and anchor:secondcol
|
// Test scanners. Writes contents:firstcol and anchor:secondcol
|
||||||
|
@ -283,7 +293,7 @@ implements RegionUnavailableListener {
|
||||||
numInserted += 2;
|
numInserted += 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
System.out.println("Write " + (vals1.length / 2) + " elapsed time: "
|
LOG.info("Write " + (vals1.length / 2) + " elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// 2. Scan from cache
|
// 2. Scan from cache
|
||||||
|
@ -321,7 +331,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + (vals1.length / 2)
|
LOG.info("Scanned " + (vals1.length / 2)
|
||||||
+ " rows from cache. Elapsed time: "
|
+ " rows from cache. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -331,7 +341,7 @@ implements RegionUnavailableListener {
|
||||||
|
|
||||||
region.flushcache();
|
region.flushcache();
|
||||||
|
|
||||||
System.out.println("Cache flush elapsed time: "
|
LOG.info("Cache flush elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// 4. Scan from disk
|
// 4. Scan from disk
|
||||||
|
@ -368,7 +378,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + (vals1.length / 2)
|
LOG.info("Scanned " + (vals1.length / 2)
|
||||||
+ " rows from disk. Elapsed time: "
|
+ " rows from disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -386,7 +396,7 @@ implements RegionUnavailableListener {
|
||||||
numInserted += 2;
|
numInserted += 2;
|
||||||
}
|
}
|
||||||
|
|
||||||
System.out.println("Write " + (vals1.length / 2) + " rows. Elapsed time: "
|
LOG.info("Write " + (vals1.length / 2) + " rows. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// 6. Scan from cache and disk
|
// 6. Scan from cache and disk
|
||||||
|
@ -423,7 +433,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + vals1.length
|
LOG.info("Scanned " + vals1.length
|
||||||
+ " rows from cache and disk. Elapsed time: "
|
+ " rows from cache and disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -433,7 +443,7 @@ implements RegionUnavailableListener {
|
||||||
|
|
||||||
region.flushcache();
|
region.flushcache();
|
||||||
|
|
||||||
System.out.println("Cache flush elapsed time: "
|
LOG.info("Cache flush elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
// 8. Scan from disk
|
// 8. Scan from disk
|
||||||
|
@ -468,7 +478,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + vals1.length
|
LOG.info("Scanned " + vals1.length
|
||||||
+ " rows from disk. Elapsed time: "
|
+ " rows from disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -506,9 +516,11 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Should have fetched " + (numInserted / 2) + " values, but fetched " + numFetched, (numInserted / 2), numFetched);
|
assertEquals("Should have fetched " + (numInserted / 2) + " values, but fetched " + numFetched, (numInserted / 2), numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + (numFetched / 2)
|
LOG.info("Scanned " + (numFetched / 2)
|
||||||
+ " rows from disk with specified start point. Elapsed time: "
|
+ " rows from disk with specified start point. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
LOG.info("scan completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
// Do a large number of writes. Disabled if not debugging because it takes a
|
// Do a large number of writes. Disabled if not debugging because it takes a
|
||||||
|
@ -517,6 +529,7 @@ implements RegionUnavailableListener {
|
||||||
|
|
||||||
private void batchWrite() throws IOException {
|
private void batchWrite() throws IOException {
|
||||||
if(! StaticTestEnvironment.debugging) {
|
if(! StaticTestEnvironment.debugging) {
|
||||||
|
LOG.info("batchWrite completed.");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -542,7 +555,7 @@ implements RegionUnavailableListener {
|
||||||
buf1.toString().getBytes(HConstants.UTF8_ENCODING));
|
buf1.toString().getBytes(HConstants.UTF8_ENCODING));
|
||||||
region.commit(writeid, System.currentTimeMillis());
|
region.commit(writeid, System.currentTimeMillis());
|
||||||
if (k > 0 && k % (N_ROWS / 100) == 0) {
|
if (k > 0 && k % (N_ROWS / 100) == 0) {
|
||||||
System.out.println("Flushing write #" + k);
|
LOG.info("Flushing write #" + k);
|
||||||
|
|
||||||
long flushStart = System.currentTimeMillis();
|
long flushStart = System.currentTimeMillis();
|
||||||
region.flushcache();
|
region.flushcache();
|
||||||
|
@ -550,51 +563,55 @@ implements RegionUnavailableListener {
|
||||||
totalFlush += (flushEnd - flushStart);
|
totalFlush += (flushEnd - flushStart);
|
||||||
|
|
||||||
if (k % (N_ROWS / 10) == 0) {
|
if (k % (N_ROWS / 10) == 0) {
|
||||||
System.out.print("Rolling log...");
|
System.err.print("Rolling log...");
|
||||||
long logStart = System.currentTimeMillis();
|
long logStart = System.currentTimeMillis();
|
||||||
log.rollWriter();
|
log.rollWriter();
|
||||||
long logEnd = System.currentTimeMillis();
|
long logEnd = System.currentTimeMillis();
|
||||||
totalLog += (logEnd - logStart);
|
totalLog += (logEnd - logStart);
|
||||||
System.out.println(" elapsed time: " + ((logEnd - logStart) / 1000.0));
|
LOG.info(" elapsed time: " + ((logEnd - logStart) / 1000.0));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
long startCompact = System.currentTimeMillis();
|
long startCompact = System.currentTimeMillis();
|
||||||
if(r.compactIfNeeded()) {
|
r.compactStores();
|
||||||
totalCompact = System.currentTimeMillis() - startCompact;
|
totalCompact = System.currentTimeMillis() - startCompact;
|
||||||
System.out.println("Region compacted - elapsedTime: " + (totalCompact / 1000.0));
|
LOG.info("Region compacted - elapsedTime: " + (totalCompact / 1000.0));
|
||||||
|
|
||||||
} else {
|
|
||||||
System.out.println("No compaction required.");
|
|
||||||
}
|
|
||||||
long endTime = System.currentTimeMillis();
|
long endTime = System.currentTimeMillis();
|
||||||
|
|
||||||
long totalElapsed = (endTime - startTime);
|
long totalElapsed = (endTime - startTime);
|
||||||
System.out.println();
|
LOG.info("");
|
||||||
System.out.println("Batch-write complete.");
|
LOG.info("Batch-write complete.");
|
||||||
System.out.println("Wrote " + N_ROWS + " rows, each of ~" + valsize + " bytes");
|
LOG.info("Wrote " + N_ROWS + " rows, each of ~" + valsize + " bytes");
|
||||||
System.out.println("Total flush-time: " + (totalFlush / 1000.0));
|
LOG.info("Total flush-time: " + (totalFlush / 1000.0));
|
||||||
System.out.println("Total compact-time: " + (totalCompact / 1000.0));
|
LOG.info("Total compact-time: " + (totalCompact / 1000.0));
|
||||||
System.out.println("Total log-time: " + (totalLog / 1000.0));
|
LOG.info("Total log-time: " + (totalLog / 1000.0));
|
||||||
System.out.println("Total time elapsed: " + (totalElapsed / 1000.0));
|
LOG.info("Total time elapsed: " + (totalElapsed / 1000.0));
|
||||||
System.out.println("Total time, rows/second: " + (N_ROWS / (totalElapsed / 1000.0)));
|
LOG.info("Total time, rows/second: " + (N_ROWS / (totalElapsed / 1000.0)));
|
||||||
System.out.println("Adjusted time (not including flush, compact, or log): " + ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0));
|
LOG.info("Adjusted time (not including flush, compact, or log): " + ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0));
|
||||||
System.out.println("Adjusted time, rows/second: " + (N_ROWS / ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0)));
|
LOG.info("Adjusted time, rows/second: " + (N_ROWS / ((totalElapsed - totalFlush - totalCompact - totalLog) / 1000.0)));
|
||||||
System.out.println();
|
LOG.info("");
|
||||||
|
|
||||||
|
LOG.info("batchWrite completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
// NOTE: This test depends on testBatchWrite succeeding
|
// NOTE: This test depends on testBatchWrite succeeding
|
||||||
private void splitAndMerge() throws IOException {
|
private void splitAndMerge() throws IOException {
|
||||||
Path oldRegionPath = r.getRegionDir();
|
Path oldRegionPath = r.getRegionDir();
|
||||||
|
Text midKey = r.compactStores();
|
||||||
|
assertNotNull(midKey);
|
||||||
long startTime = System.currentTimeMillis();
|
long startTime = System.currentTimeMillis();
|
||||||
HRegion subregions[] = r.splitRegion(this);
|
HRegion subregions[] = r.splitRegion(this, midKey);
|
||||||
if (subregions != null) {
|
if (subregions != null) {
|
||||||
System.out.println("Split region elapsed time: "
|
LOG.info("Split region elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
assertEquals("Number of subregions", subregions.length, 2);
|
assertEquals("Number of subregions", subregions.length, 2);
|
||||||
|
|
||||||
|
for (int i = 0; i < subregions.length; i++) {
|
||||||
|
subregions[i] = openClosedRegion(subregions[i]);
|
||||||
|
subregions[i].compactStores();
|
||||||
|
}
|
||||||
|
|
||||||
// Now merge it back together
|
// Now merge it back together
|
||||||
|
|
||||||
Path oldRegion1 = subregions[0].getRegionDir();
|
Path oldRegion1 = subregions[0].getRegionDir();
|
||||||
|
@ -602,12 +619,13 @@ implements RegionUnavailableListener {
|
||||||
startTime = System.currentTimeMillis();
|
startTime = System.currentTimeMillis();
|
||||||
r = HRegion.mergeAdjacent(subregions[0], subregions[1]);
|
r = HRegion.mergeAdjacent(subregions[0], subregions[1]);
|
||||||
region = new HRegionIncommon(r);
|
region = new HRegionIncommon(r);
|
||||||
System.out.println("Merge regions elapsed time: "
|
LOG.info("Merge regions elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
fs.delete(oldRegion1);
|
fs.delete(oldRegion1);
|
||||||
fs.delete(oldRegion2);
|
fs.delete(oldRegion2);
|
||||||
fs.delete(oldRegionPath);
|
fs.delete(oldRegionPath);
|
||||||
}
|
}
|
||||||
|
LOG.info("splitAndMerge completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -668,7 +686,7 @@ implements RegionUnavailableListener {
|
||||||
anchorFetched++;
|
anchorFetched++;
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
System.out.println("UNEXPECTED COLUMN " + col);
|
LOG.info("UNEXPECTED COLUMN " + col);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
curVals.clear();
|
curVals.clear();
|
||||||
|
@ -677,7 +695,7 @@ implements RegionUnavailableListener {
|
||||||
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
|
assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
|
||||||
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
|
assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + NUM_VALS
|
LOG.info("Scanned " + NUM_VALS
|
||||||
+ " rows from disk. Elapsed time: "
|
+ " rows from disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -720,7 +738,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
assertEquals("Inserted " + numInserted + " values, but fetched " + numFetched, numInserted, numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + (numFetched / 2)
|
LOG.info("Scanned " + (numFetched / 2)
|
||||||
+ " rows from disk. Elapsed time: "
|
+ " rows from disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -753,7 +771,7 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + N_ROWS + " values, but fetched " + numFetched, N_ROWS, numFetched);
|
assertEquals("Inserted " + N_ROWS + " values, but fetched " + numFetched, N_ROWS, numFetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + N_ROWS
|
LOG.info("Scanned " + N_ROWS
|
||||||
+ " rows from disk. Elapsed time: "
|
+ " rows from disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
|
@ -785,37 +803,14 @@ implements RegionUnavailableListener {
|
||||||
}
|
}
|
||||||
assertEquals("Inserted " + (NUM_VALS + numInserted/2) + " values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched);
|
assertEquals("Inserted " + (NUM_VALS + numInserted/2) + " values, but fetched " + fetched, (NUM_VALS + numInserted/2), fetched);
|
||||||
|
|
||||||
System.out.println("Scanned " + fetched
|
LOG.info("Scanned " + fetched
|
||||||
+ " rows from disk. Elapsed time: "
|
+ " rows from disk. Elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
|
||||||
} finally {
|
} finally {
|
||||||
s.close();
|
s.close();
|
||||||
}
|
}
|
||||||
|
LOG.info("read completed.");
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void deleteFile(File f) {
|
|
||||||
if(f.isDirectory()) {
|
|
||||||
File[] children = f.listFiles();
|
|
||||||
for(int i = 0; i < children.length; i++) {
|
|
||||||
deleteFile(children[i]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
f.delete();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void cleanup() {
|
|
||||||
try {
|
|
||||||
r.close();
|
|
||||||
r = null;
|
|
||||||
log.closeAndDelete();
|
|
||||||
log = null;
|
|
||||||
} catch (IOException e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
}
|
|
||||||
|
|
||||||
// Delete all the DFS files
|
|
||||||
|
|
||||||
deleteFile(new File(System.getProperty("test.build.data"), "dfs"));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,16 +24,15 @@ import java.util.TreeMap;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.log4j.Level;
|
import org.apache.log4j.Level;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
import org.apache.hadoop.hbase.MultiRegionTable;
|
import org.apache.hadoop.hbase.MultiRegionTable;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.HScannerInterface;
|
import org.apache.hadoop.hbase.HScannerInterface;
|
||||||
import org.apache.hadoop.hbase.StaticTestEnvironment;
|
|
||||||
import org.apache.hadoop.hbase.io.Cell;
|
import org.apache.hadoop.hbase.io.Cell;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -75,6 +74,7 @@ public class TestSplit extends MultiRegionTable {
|
||||||
HRegion region = null;
|
HRegion region = null;
|
||||||
try {
|
try {
|
||||||
HTableDescriptor htd = createTableDescriptor(getName());
|
HTableDescriptor htd = createTableDescriptor(getName());
|
||||||
|
htd.addFamily(new HColumnDescriptor(COLFAMILY_NAME3));
|
||||||
region = createNewHRegion(htd, null, null);
|
region = createNewHRegion(htd, null, null);
|
||||||
basicSplit(region);
|
basicSplit(region);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -88,9 +88,9 @@ public class TestSplit extends MultiRegionTable {
|
||||||
private void basicSplit(final HRegion region) throws Exception {
|
private void basicSplit(final HRegion region) throws Exception {
|
||||||
addContent(region, COLFAMILY_NAME3);
|
addContent(region, COLFAMILY_NAME3);
|
||||||
region.flushcache();
|
region.flushcache();
|
||||||
Text midkey = new Text();
|
Text midkey = region.compactStores();
|
||||||
assertTrue(region.needsSplit(midkey));
|
assertNotNull(midkey);
|
||||||
HRegion [] regions = split(region);
|
HRegion [] regions = split(region, midkey);
|
||||||
try {
|
try {
|
||||||
// Need to open the regions.
|
// Need to open the regions.
|
||||||
// TODO: Add an 'open' to HRegion... don't do open by constructing
|
// TODO: Add an 'open' to HRegion... don't do open by constructing
|
||||||
|
@ -106,17 +106,9 @@ public class TestSplit extends MultiRegionTable {
|
||||||
assertScan(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
|
assertScan(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
|
||||||
assertScan(regions[1], COLFAMILY_NAME3, midkey);
|
assertScan(regions[1], COLFAMILY_NAME3, midkey);
|
||||||
// Now prove can't split regions that have references.
|
// Now prove can't split regions that have references.
|
||||||
Text[] midkeys = new Text[regions.length];
|
|
||||||
for (int i = 0; i < regions.length; i++) {
|
for (int i = 0; i < regions.length; i++) {
|
||||||
midkeys[i] = new Text();
|
|
||||||
// Even after above splits, still needs split but after splits its
|
|
||||||
// unsplitable because biggest store file is reference. References
|
|
||||||
// make the store unsplittable, until something bigger comes along.
|
|
||||||
assertFalse(regions[i].needsSplit(midkeys[i]));
|
|
||||||
// Add so much data to this region, we create a store file that is >
|
// Add so much data to this region, we create a store file that is >
|
||||||
// than
|
// than one of our unsplitable references. it will.
|
||||||
// one of our unsplitable references.
|
|
||||||
// it will.
|
|
||||||
for (int j = 0; j < 2; j++) {
|
for (int j = 0; j < 2; j++) {
|
||||||
addContent(regions[i], COLFAMILY_NAME3);
|
addContent(regions[i], COLFAMILY_NAME3);
|
||||||
}
|
}
|
||||||
|
@ -125,32 +117,25 @@ public class TestSplit extends MultiRegionTable {
|
||||||
regions[i].flushcache();
|
regions[i].flushcache();
|
||||||
}
|
}
|
||||||
|
|
||||||
// Assert that even if one store file is larger than a reference, the
|
Text[] midkeys = new Text[regions.length];
|
||||||
// region is still deemed unsplitable (Can't split region if references
|
|
||||||
// presen).
|
|
||||||
for (int i = 0; i < regions.length; i++) {
|
|
||||||
midkeys[i] = new Text();
|
|
||||||
// Even after above splits, still needs split but after splits its
|
|
||||||
// unsplitable because biggest store file is reference. References
|
|
||||||
// make the store unsplittable, until something bigger comes along.
|
|
||||||
assertFalse(regions[i].needsSplit(midkeys[i]));
|
|
||||||
}
|
|
||||||
|
|
||||||
// To make regions splitable force compaction.
|
// To make regions splitable force compaction.
|
||||||
for (int i = 0; i < regions.length; i++) {
|
for (int i = 0; i < regions.length; i++) {
|
||||||
regions[i].compactStores();
|
midkeys[i] = regions[i].compactStores();
|
||||||
}
|
}
|
||||||
|
|
||||||
TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
|
TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
|
||||||
// Split these two daughter regions so then I'll have 4 regions. Will
|
// Split these two daughter regions so then I'll have 4 regions. Will
|
||||||
// split because added data above.
|
// split because added data above.
|
||||||
for (int i = 0; i < regions.length; i++) {
|
for (int i = 0; i < regions.length; i++) {
|
||||||
HRegion[] rs = split(regions[i]);
|
HRegion[] rs = null;
|
||||||
|
if (midkeys[i] != null) {
|
||||||
|
rs = split(regions[i], midkeys[i]);
|
||||||
for (int j = 0; j < rs.length; j++) {
|
for (int j = 0; j < rs.length; j++) {
|
||||||
sortedMap.put(rs[j].getRegionName().toString(),
|
sortedMap.put(rs[j].getRegionName().toString(),
|
||||||
openClosedRegion(rs[j]));
|
openClosedRegion(rs[j]));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
LOG.info("Made 4 regions");
|
LOG.info("Made 4 regions");
|
||||||
// The splits should have been even. Test I can get some arbitrary row out
|
// The splits should have been even. Test I can get some arbitrary row out
|
||||||
// of each.
|
// of each.
|
||||||
|
@ -219,12 +204,11 @@ public class TestSplit extends MultiRegionTable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private HRegion [] split(final HRegion r) throws IOException {
|
private HRegion [] split(final HRegion r, final Text midKey)
|
||||||
Text midKey = new Text();
|
throws IOException {
|
||||||
assertTrue(r.needsSplit(midKey));
|
|
||||||
// Assert can get mid key from passed region.
|
// Assert can get mid key from passed region.
|
||||||
assertGet(r, COLFAMILY_NAME3, midKey);
|
assertGet(r, COLFAMILY_NAME3, midKey);
|
||||||
HRegion [] regions = r.splitRegion(null);
|
HRegion [] regions = r.splitRegion(null, midKey);
|
||||||
assertEquals(regions.length, 2);
|
assertEquals(regions.length, 2);
|
||||||
return regions;
|
return regions;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,20 +19,12 @@
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.TreeMap;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
|
import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
|
||||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
|
||||||
import org.apache.hadoop.hbase.HScannerInterface;
|
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.io.Cell;
|
import org.apache.hadoop.hbase.HBaseClusterTestCase;
|
||||||
import org.apache.hadoop.hbase.HBaseTestCase;
|
|
||||||
import org.apache.hadoop.hbase.TimestampTestBase;
|
import org.apache.hadoop.hbase.TimestampTestBase;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
|
@ -43,7 +35,7 @@ import org.apache.commons.logging.LogFactory;
|
||||||
* tests same in presence of deletes. Test cores are written so can be
|
* 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.
|
* run against an HRegion and against an HTable: i.e. both local and remote.
|
||||||
*/
|
*/
|
||||||
public class TestTimestamp extends HBaseTestCase {
|
public class TestTimestamp extends HBaseClusterTestCase {
|
||||||
private static final Log LOG =
|
private static final Log LOG =
|
||||||
LogFactory.getLog(TestTimestamp.class.getName());
|
LogFactory.getLog(TestTimestamp.class.getName());
|
||||||
|
|
||||||
|
@ -51,11 +43,6 @@ public class TestTimestamp extends HBaseTestCase {
|
||||||
private static final Text COLUMN = new Text(COLUMN_NAME);
|
private static final Text COLUMN = new Text(COLUMN_NAME);
|
||||||
private static final int VERSIONS = 3;
|
private static final int VERSIONS = 3;
|
||||||
|
|
||||||
/** constructor */
|
|
||||||
public TestTimestamp() {
|
|
||||||
super();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that delete works according to description in <a
|
* Test that delete works according to description in <a
|
||||||
* href="https://issues.apache.org/jira/browse/HADOOP-1784">hadoop-1784</a>.
|
* href="https://issues.apache.org/jira/browse/HADOOP-1784">hadoop-1784</a>.
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.StaticTestEnvironment;
|
import org.apache.hadoop.hbase.StaticTestEnvironment;
|
||||||
import org.apache.hadoop.hbase.io.BatchUpdate;
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
|
import org.apache.hadoop.hbase.io.Cell;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
|
||||||
/** Test stand alone merge tool that can merge arbitrary regions */
|
/** Test stand alone merge tool that can merge arbitrary regions */
|
||||||
|
@ -54,6 +55,7 @@ public class TestMergeTool extends HBaseTestCase {
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
|
this.conf.set("hbase.hstore.compactionThreshold", "2");
|
||||||
|
|
||||||
// Create table description
|
// Create table description
|
||||||
|
|
||||||
|
@ -237,7 +239,9 @@ public class TestMergeTool extends HBaseTestCase {
|
||||||
|
|
||||||
for (int i = 0; i < 3 ; i++) {
|
for (int i = 0; i < 3 ; i++) {
|
||||||
for (int j = 0; j < rows[i].length; j++) {
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
byte[] bytes = merged.get(rows[i][j], COLUMN_NAME).getValue();
|
Cell cell = merged.get(rows[i][j], COLUMN_NAME);
|
||||||
|
assertNotNull(cell);
|
||||||
|
byte[] bytes = cell.getValue();
|
||||||
assertNotNull(bytes);
|
assertNotNull(bytes);
|
||||||
Text value = new Text(bytes);
|
Text value = new Text(bytes);
|
||||||
assertTrue(value.equals(rows[i][j]));
|
assertTrue(value.equals(rows[i][j]));
|
||||||
|
|
Loading…
Reference in New Issue