HBASE-3796 Per-Store Entries in Compaction Queue

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1098021 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Nicolas Spiegelberg 2011-04-29 23:07:52 +00:00
parent 15edb7b8e4
commit 47fe311acd
7 changed files with 234 additions and 174 deletions

View File

@ -194,6 +194,7 @@ Release 0.91.0 - Unreleased
HBASE-3812 Tidy up naming consistency and documentation in coprocessor HBASE-3812 Tidy up naming consistency and documentation in coprocessor
framework (Mingjie Lai) framework (Mingjie Lai)
HBASE-1512 Support aggregate functions (Himanshu Vashishtha) HBASE-1512 Support aggregate functions (Himanshu Vashishtha)
HBASE-3796 Per-Store Enties in Compaction Queue
TASKS TASKS
HBASE-3559 Move report of split to master OFF the heartbeat channel HBASE-3559 Move report of split to master OFF the heartbeat channel

View File

@ -27,6 +27,8 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
/** /**
@ -70,15 +72,17 @@ public class CompactSplitThread extends Thread implements CompactionRequestor {
@Override @Override
public void run() { public void run() {
while (!this.server.isStopped()) { while (!this.server.isStopped()) {
CompactionRequest compactionRequest = null;
HRegion r = null; HRegion r = null;
try { try {
r = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS); compactionRequest = compactionQueue.poll(this.frequency, TimeUnit.MILLISECONDS);
if (r != null) { if (compactionRequest != null) {
lock.lock(); lock.lock();
try { try {
if(!this.server.isStopped()) { if(!this.server.isStopped()) {
// Don't interrupt us while we are working // Don't interrupt us while we are working
byte [] midKey = r.compactStores(); r = compactionRequest.getHRegion();
byte [] midKey = r.compactStore(compactionRequest.getStore());
if (r.getLastCompactInfo() != null) { // compaction aborted? if (r.getLastCompactInfo() != null) { // compaction aborted?
this.server.getMetrics().addCompaction(r.getLastCompactInfo()); this.server.getMetrics().addCompaction(r.getLastCompactInfo());
} }
@ -115,7 +119,9 @@ public class CompactSplitThread extends Thread implements CompactionRequestor {
public synchronized void requestCompaction(final HRegion r, public synchronized void requestCompaction(final HRegion r,
final String why) { final String why) {
requestCompaction(r, false, why, r.getCompactPriority()); for(Store s : r.getStores().values()) {
requestCompaction(r, s, false, why, s.getCompactPriority());
}
} }
public synchronized void requestCompaction(final HRegion r, public synchronized void requestCompaction(final HRegion r,
@ -123,23 +129,33 @@ public class CompactSplitThread extends Thread implements CompactionRequestor {
requestCompaction(r, false, why, p); requestCompaction(r, false, why, p);
} }
public synchronized void requestCompaction(final HRegion r,
final boolean force, final String why, int p) {
for(Store s : r.getStores().values()) {
requestCompaction(r, s, force, why, p);
}
}
/** /**
* @param r HRegion store belongs to * @param r HRegion store belongs to
* @param force Whether next compaction should be major * @param force Whether next compaction should be major
* @param why Why compaction requested -- used in debug messages * @param why Why compaction requested -- used in debug messages
*/ */
public synchronized void requestCompaction(final HRegion r, public synchronized void requestCompaction(final HRegion r, final Store s,
final boolean force, final String why, int priority) { final boolean force, final String why, int priority) {
if (this.server.isStopped()) { if (this.server.isStopped()) {
return; return;
} }
// tell the region to major-compact (and don't downgrade it) // tell the region to major-compact (and don't downgrade it)
if (force) { if (force) {
r.setForceMajorCompaction(force); s.setForceMajorCompaction(force);
} }
if (compactionQueue.add(r, priority) && LOG.isDebugEnabled()) { CompactionRequest compactionRequest = new CompactionRequest(r, s, priority);
if (compactionQueue.add(compactionRequest) && LOG.isDebugEnabled()) {
LOG.debug("Compaction " + (force? "(major) ": "") + LOG.debug("Compaction " + (force? "(major) ": "") +
"requested for " + r.getRegionNameAsString() + "requested for region " + r.getRegionNameAsString() +
"/" + r.getRegionInfo().getEncodedName() +
", store " + s +
(why != null && !why.isEmpty()? " because " + why: "") + (why != null && !why.isEmpty()? " because " + why: "") +
"; priority=" + priority + ", compaction queue size=" + compactionQueue.size()); "; priority=" + priority + ", compaction queue size=" + compactionQueue.size());
} }

View File

@ -788,6 +788,20 @@ public class HRegion implements HeapSize { // , Writable{
return compactStores(); return compactStores();
} }
/**
* Compact all the stores and return the split key of the first store that needs
* to be split.
*/
public byte[] compactStores() throws IOException {
byte[] splitRow = null;
for(Store s : getStores().values()) {
if(splitRow == null) {
splitRow = compactStore(s);
}
}
return splitRow;
}
/* /*
* Called by compaction thread and after region is opened to compact the * Called by compaction thread and after region is opened to compact the
* HStores if necessary. * HStores if necessary.
@ -802,7 +816,7 @@ public class HRegion implements HeapSize { // , Writable{
* @return split row if split is needed * @return split row if split is needed
* @throws IOException e * @throws IOException e
*/ */
public byte [] compactStores() throws IOException { public byte [] compactStore(Store store) throws IOException {
if (this.closing.get()) { if (this.closing.get()) {
LOG.debug("Skipping compaction on " + this + " because closing"); LOG.debug("Skipping compaction on " + this + " because closing");
return null; return null;
@ -836,17 +850,13 @@ public class HRegion implements HeapSize { // , Writable{
long startTime = EnvironmentEdgeManager.currentTimeMillis(); long startTime = EnvironmentEdgeManager.currentTimeMillis();
doRegionCompactionPrep(); doRegionCompactionPrep();
long lastCompactSize = 0; long lastCompactSize = 0;
long maxSize = -1;
boolean completed = false; boolean completed = false;
try { try {
for (Store store: stores.values()) {
final Store.StoreSize ss = store.compact(); final Store.StoreSize ss = store.compact();
lastCompactSize += store.getLastCompactSize(); lastCompactSize += store.getLastCompactSize();
if (ss != null && ss.getSize() > maxSize) { if (ss != null) {
maxSize = ss.getSize();
splitRow = ss.getSplitRow(); splitRow = ss.getSplitRow();
} }
}
completed = true; completed = true;
} catch (InterruptedIOException iioe) { } catch (InterruptedIOException iioe) {
LOG.info("compaction interrupted by user: ", iioe); LOG.info("compaction interrupted by user: ", iioe);
@ -2224,6 +2234,10 @@ public class HRegion implements HeapSize { // , Writable{
return this.stores.get(column); return this.stores.get(column);
} }
public Map<byte[], Store> getStores() {
return this.stores;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Support code // Support code
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
@ -2403,12 +2417,12 @@ public class HRegion implements HeapSize { // , Writable{
if (!(o instanceof HRegion)) { if (!(o instanceof HRegion)) {
return false; return false;
} }
return Bytes.equals(this.regionInfo.getRegionName(), ((HRegion)o).regionInfo.getRegionName()); return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
} }
@Override @Override
public int hashCode() { public int hashCode() {
return Bytes.hashCode(this.regionInfo.getRegionName()); return Bytes.hashCode(this.getRegionName());
} }
@Override @Override

View File

@ -20,132 +20,69 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.util.Collection; import java.util.Collection;
import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
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.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.Pair;
/** /**
* This class delegates to the BlockingQueue but wraps all HRegions in * This class delegates to the BlockingQueue but wraps all Stores in
* compaction requests that hold the priority and the date requested. * compaction requests that hold the priority and the date requested.
* *
* Implementation Note: With an elevation time of -1 there is the potential for * Implementation Note: With an elevation time of -1 there is the potential for
* starvation of the lower priority compaction requests as long as there is a * starvation of the lower priority compaction requests as long as there is a
* constant stream of high priority requests. * constant stream of high priority requests.
*/ */
public class PriorityCompactionQueue implements BlockingQueue<HRegion> { public class PriorityCompactionQueue implements BlockingQueue<CompactionRequest> {
static final Log LOG = LogFactory.getLog(PriorityCompactionQueue.class); static final Log LOG = LogFactory.getLog(PriorityCompactionQueue.class);
/**
* This class represents a compaction request and holds the region, priority,
* and time submitted.
*/
private class CompactionRequest implements Comparable<CompactionRequest> {
private final HRegion r;
private final int p;
private final Date date;
public CompactionRequest(HRegion r, int p) {
this(r, p, null);
}
public CompactionRequest(HRegion r, int p, Date d) {
if (r == null) {
throw new NullPointerException("HRegion cannot be null");
}
if (d == null) {
d = new Date();
}
this.r = r;
this.p = p;
this.date = d;
}
/**
* This function will define where in the priority queue the request will
* end up. Those with the highest priorities will be first. When the
* priorities are the same it will It will first compare priority then date
* to maintain a FIFO functionality.
*
* <p>Note: The date is only accurate to the millisecond which means it is
* possible that two requests were inserted into the queue within a
* millisecond. When that is the case this function will break the tie
* arbitrarily.
*/
@Override
public int compareTo(CompactionRequest request) {
//NOTE: The head of the priority queue is the least element
if (this.equals(request)) {
return 0; //they are the same request
}
int compareVal;
compareVal = p - request.p; //compare priority
if (compareVal != 0) {
return compareVal;
}
compareVal = date.compareTo(request.date);
if (compareVal != 0) {
return compareVal;
}
//break the tie arbitrarily
return -1;
}
/** Gets the HRegion for the request */
HRegion getHRegion() {
return r;
}
/** Gets the priority for the request */
int getPriority() {
return p;
}
public String toString() {
return "regionName=" + r.getRegionNameAsString() +
", priority=" + p + ", date=" + date;
}
}
/** The actual blocking queue we delegate to */ /** The actual blocking queue we delegate to */
protected final BlockingQueue<CompactionRequest> queue = protected final BlockingQueue<CompactionRequest> queue =
new PriorityBlockingQueue<CompactionRequest>(); new PriorityBlockingQueue<CompactionRequest>();
/** Hash map of the HRegions contained within the Compaction Queue */ /** Hash map of the Stores contained within the Compaction Queue */
private final HashMap<HRegion, CompactionRequest> regionsInQueue = private final HashMap<Pair<HRegion, Store>, CompactionRequest> storesInQueue =
new HashMap<HRegion, CompactionRequest>(); new HashMap<Pair<HRegion, Store>, CompactionRequest>();
/** Creates a new PriorityCompactionQueue with no priority elevation time */ /** Creates a new PriorityCompactionQueue with no priority elevation time */
public PriorityCompactionQueue() { public PriorityCompactionQueue() {
LOG.debug("Create PriorityCompactionQueue"); LOG.debug("Create PriorityCompactionQueue");
} }
/** If the region is not already in the queue it will add it and return a protected Pair<HRegion, Store> toPair(CompactionRequest cr) {
return Pair.newPair(cr.getHRegion(), cr.getStore());
}
/** If the store is not already in the queue it will add it and return a
* new compaction request object. If it is already present in the queue * new compaction request object. If it is already present in the queue
* then it will return null. * then it will return null.
* @param p If null it will use the default priority * @param p If null it will use the default priority
* @return returns a compaction request if it isn't already in the queue * @return returns a compaction request if it isn't already in the queue
*/ */
protected CompactionRequest addToRegionsInQueue(HRegion r, int p) { protected CompactionRequest addToCompactionQueue(CompactionRequest newRequest) {
CompactionRequest queuedRequest = null; CompactionRequest queuedRequest = null;
CompactionRequest newRequest = new CompactionRequest(r, p); synchronized (storesInQueue) {
synchronized (regionsInQueue) { queuedRequest = storesInQueue.get(toPair(newRequest));
queuedRequest = regionsInQueue.get(r);
if (queuedRequest == null || if (queuedRequest == null ||
newRequest.getPriority() < queuedRequest.getPriority()) { newRequest.getPriority() < queuedRequest.getPriority()) {
LOG.trace("Inserting region in queue. " + newRequest); String reason = "";
regionsInQueue.put(r, newRequest); if (queuedRequest != null) {
if (newRequest.getPriority() < queuedRequest.getPriority()) {
reason = "Reason : priority changed from " +
queuedRequest.getPriority() + " to " +
newRequest.getPriority() + ". ";
}
}
LOG.debug("Inserting store in queue. " + reason + newRequest);
storesInQueue.put(toPair(newRequest), newRequest);
} else { } else {
LOG.trace("Region already in queue, skipping. Queued: " + queuedRequest + LOG.debug("Store already in queue, skipping. Queued: " + queuedRequest +
", requested: " + newRequest); ", requested: " + newRequest);
newRequest = null; // It is already present so don't add it newRequest = null; // It is already present so don't add it
} }
@ -159,33 +96,34 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
return newRequest; return newRequest;
} }
/** Removes the request from the regions in queue /** Removes the request from the stores in queue
* @param remove * @param remove
*/ */
protected CompactionRequest removeFromRegionsInQueue(CompactionRequest remove) { protected CompactionRequest removeFromQueue(CompactionRequest c) {
if (remove == null) return null; if (c == null) return null;
synchronized (regionsInQueue) { synchronized (storesInQueue) {
CompactionRequest cr = null; CompactionRequest cr = storesInQueue.remove(toPair(c));
cr = regionsInQueue.remove(remove.getHRegion()); if (cr != null && !cr.equals(c))
if (cr != null && !cr.equals(remove))
{ {
//Because we don't synchronize across both this.regionsInQueue and this.queue //Because we don't synchronize across both this.regionsInQueue and this.queue
//a rare race condition exists where a higher priority compaction request replaces //a rare race condition exists where a higher priority compaction request replaces
//the lower priority request in this.regionsInQueue but the lower priority request //the lower priority request in this.regionsInQueue but the lower priority request
//is taken off this.queue before the higher can be added to this.queue. //is taken off this.queue before the higher can be added to this.queue.
//So if we didn't remove what we were expecting we put it back on. //So if we didn't remove what we were expecting we put it back on.
regionsInQueue.put(cr.getHRegion(), cr); storesInQueue.put(toPair(cr), cr);
} }
if (cr == null) { if (cr == null) {
LOG.warn("Removed a region it couldn't find in regionsInQueue: " + remove.getHRegion()); LOG.warn("Removed a compaction request it couldn't find in storesInQueue: " +
"region = " + c.getHRegion() + ", store = " + c.getStore());
} }
return cr; return cr;
} }
} }
public boolean add(HRegion e, int p) { @Override
CompactionRequest request = this.addToRegionsInQueue(e, p); public boolean add(CompactionRequest e) {
CompactionRequest request = this.addToCompactionQueue(e);
if (request != null) { if (request != null) {
boolean result = queue.add(request); boolean result = queue.add(request);
return result; return result;
@ -195,68 +133,50 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
} }
@Override @Override
public boolean add(HRegion e) { public boolean offer(CompactionRequest e) {
return add(e, e.getCompactPriority()); CompactionRequest request = this.addToCompactionQueue(e);
}
public boolean offer(HRegion e, int p) {
CompactionRequest request = this.addToRegionsInQueue(e, p);
return (request != null)? queue.offer(request): false; return (request != null)? queue.offer(request): false;
} }
@Override @Override
public boolean offer(HRegion e) { public void put(CompactionRequest e) throws InterruptedException {
return offer(e, e.getCompactPriority()); CompactionRequest request = this.addToCompactionQueue(e);
}
public void put(HRegion e, int p) throws InterruptedException {
CompactionRequest request = this.addToRegionsInQueue(e, p);
if (request != null) { if (request != null) {
queue.put(request); queue.put(request);
} }
} }
@Override @Override
public void put(HRegion e) throws InterruptedException { public boolean offer(CompactionRequest e, long timeout, TimeUnit unit)
put(e, e.getCompactPriority());
}
public boolean offer(HRegion e, int p, long timeout, TimeUnit unit)
throws InterruptedException { throws InterruptedException {
CompactionRequest request = this.addToRegionsInQueue(e, p); CompactionRequest request = this.addToCompactionQueue(e);
return (request != null)? queue.offer(request, timeout, unit): false; return (request != null)? queue.offer(request, timeout, unit): false;
} }
@Override @Override
public boolean offer(HRegion e, long timeout, TimeUnit unit) public CompactionRequest take() throws InterruptedException {
throws InterruptedException {
return offer(e, e.getCompactPriority(), timeout, unit);
}
@Override
public HRegion take() throws InterruptedException {
CompactionRequest cr = queue.take(); CompactionRequest cr = queue.take();
if (cr != null) { if (cr != null) {
removeFromRegionsInQueue(cr); removeFromQueue(cr);
return cr.getHRegion(); return cr;
} }
return null; return null;
} }
@Override @Override
public HRegion poll(long timeout, TimeUnit unit) throws InterruptedException { public CompactionRequest poll(long timeout, TimeUnit unit) throws InterruptedException {
CompactionRequest cr = queue.poll(timeout, unit); CompactionRequest cr = queue.poll(timeout, unit);
if (cr != null) { if (cr != null) {
removeFromRegionsInQueue(cr); removeFromQueue(cr);
return cr.getHRegion(); return cr;
} }
return null; return null;
} }
@Override @Override
public boolean remove(Object r) { public boolean remove(Object o) {
if (r instanceof CompactionRequest) { if (o instanceof CompactionRequest) {
CompactionRequest cr = removeFromRegionsInQueue((CompactionRequest) r); CompactionRequest cr = removeFromQueue((CompactionRequest) o);
if (cr != null) { if (cr != null) {
return queue.remove(cr); return queue.remove(cr);
} }
@ -266,21 +186,21 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
} }
@Override @Override
public HRegion remove() { public CompactionRequest remove() {
CompactionRequest cr = queue.remove(); CompactionRequest cr = queue.remove();
if (cr != null) { if (cr != null) {
removeFromRegionsInQueue(cr); removeFromQueue(cr);
return cr.getHRegion(); return cr;
} }
return null; return null;
} }
@Override @Override
public HRegion poll() { public CompactionRequest poll() {
CompactionRequest cr = queue.poll(); CompactionRequest cr = queue.poll();
if (cr != null) { if (cr != null) {
removeFromRegionsInQueue(cr); removeFromQueue(cr);
return cr.getHRegion(); return cr;
} }
return null; return null;
} }
@ -292,9 +212,9 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
@Override @Override
public boolean contains(Object r) { public boolean contains(Object r) {
if (r instanceof HRegion) { if (r instanceof CompactionRequest) {
synchronized (regionsInQueue) { synchronized (storesInQueue) {
return regionsInQueue.containsKey((HRegion) r); return storesInQueue.containsKey(toPair((CompactionRequest) r));
} }
} else if (r instanceof CompactionRequest) { } else if (r instanceof CompactionRequest) {
return queue.contains(r); return queue.contains(r);
@ -303,15 +223,15 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
} }
@Override @Override
public HRegion element() { public CompactionRequest element() {
CompactionRequest cr = queue.element(); CompactionRequest cr = queue.element();
return (cr != null)? cr.getHRegion(): null; return (cr != null)? cr: null;
} }
@Override @Override
public HRegion peek() { public CompactionRequest peek() {
CompactionRequest cr = queue.peek(); CompactionRequest cr = queue.peek();
return (cr != null)? cr.getHRegion(): null; return (cr != null)? cr: null;
} }
@Override @Override
@ -326,14 +246,14 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
@Override @Override
public void clear() { public void clear() {
regionsInQueue.clear(); storesInQueue.clear();
queue.clear(); queue.clear();
} }
// Unimplemented methods, collection methods // Unimplemented methods, collection methods
@Override @Override
public Iterator<HRegion> iterator() { public Iterator<CompactionRequest> iterator() {
throw new UnsupportedOperationException("Not supported."); throw new UnsupportedOperationException("Not supported.");
} }
@ -353,7 +273,7 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
} }
@Override @Override
public boolean addAll(Collection<? extends HRegion> c) { public boolean addAll(Collection<? extends CompactionRequest> c) {
throw new UnsupportedOperationException("Not supported."); throw new UnsupportedOperationException("Not supported.");
} }
@ -368,12 +288,12 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
} }
@Override @Override
public int drainTo(Collection<? super HRegion> c) { public int drainTo(Collection<? super CompactionRequest> c) {
throw new UnsupportedOperationException("Not supported."); throw new UnsupportedOperationException("Not supported.");
} }
@Override @Override
public int drainTo(Collection<? super HRegion> c, int maxElements) { public int drainTo(Collection<? super CompactionRequest> c, int maxElements) {
throw new UnsupportedOperationException("Not supported."); throw new UnsupportedOperationException("Not supported.");
} }
} }

View File

@ -1479,7 +1479,7 @@ public class Store implements HeapSize {
/** /**
* @return The priority that this store should have in the compaction queue * @return The priority that this store should have in the compaction queue
*/ */
int getCompactPriority() { public int getCompactPriority() {
return this.blockingStoreFileCount - this.storefiles.size(); return this.blockingStoreFileCount - this.storefiles.size();
} }

View File

@ -0,0 +1,103 @@
package org.apache.hadoop.hbase.regionserver.compactions;
import java.util.Date;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
/**
* This class represents a compaction request and holds the region, priority,
* and time submitted.
*/
public class CompactionRequest implements Comparable<CompactionRequest> {
static final Log LOG = LogFactory.getLog(CompactionRequest.class);
private final HRegion r;
private final Store s;
private int p;
private final Date date;
public CompactionRequest(HRegion r, Store s) {
this(r, s, s.getCompactPriority());
}
public CompactionRequest(HRegion r, Store s, int p) {
this(r, s, p, null);
}
public CompactionRequest(HRegion r, Store s, int p, Date d) {
if (r == null) {
throw new NullPointerException("HRegion cannot be null");
}
if (d == null) {
d = new Date();
}
this.r = r;
this.s = s;
this.p = p;
this.date = d;
}
/**
* This function will define where in the priority queue the request will
* end up. Those with the highest priorities will be first. When the
* priorities are the same it will It will first compare priority then date
* to maintain a FIFO functionality.
*
* <p>Note: The date is only accurate to the millisecond which means it is
* possible that two requests were inserted into the queue within a
* millisecond. When that is the case this function will break the tie
* arbitrarily.
*/
@Override
public int compareTo(CompactionRequest request) {
//NOTE: The head of the priority queue is the least element
if (this.equals(request)) {
return 0; //they are the same request
}
int compareVal;
compareVal = p - request.p; //compare priority
if (compareVal != 0) {
return compareVal;
}
compareVal = date.compareTo(request.date);
if (compareVal != 0) {
return compareVal;
}
//break the tie arbitrarily
return -1;
}
/** Gets the HRegion for the request */
public HRegion getHRegion() {
return r;
}
/** Gets the Store for the request */
public Store getStore() {
return s;
}
/** Gets the priority for the request */
public int getPriority() {
return p;
}
/** Gets the priority for the request */
public void setPriority(int p) {
this.p = p;
}
public String toString() {
return "regionName=" + r.getRegionNameAsString() +
((s == null) ? ""
: "storeName = " + new String(s.getFamily().getName())) +
", priority=" + p + ", date=" + date;
}
}

View File

@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
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.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -61,6 +63,10 @@ public class TestPriorityCompactionQueue {
return "[DummyHRegion " + name + "]"; return "[DummyHRegion " + name + "]";
} }
public byte[] getRegionName() {
return Bytes.toBytes(name);
}
public String getRegionNameAsString() { public String getRegionNameAsString() {
return name; return name;
} }
@ -68,7 +74,7 @@ public class TestPriorityCompactionQueue {
protected void getAndCheckRegion(PriorityCompactionQueue pq, protected void getAndCheckRegion(PriorityCompactionQueue pq,
HRegion checkRegion) { HRegion checkRegion) {
HRegion r = pq.remove(); HRegion r = pq.remove().getHRegion();
if (r != checkRegion) { if (r != checkRegion) {
Assert.assertTrue("Didn't get expected " + checkRegion + " got " + r, r Assert.assertTrue("Didn't get expected " + checkRegion + " got " + r, r
.equals(checkRegion)); .equals(checkRegion));
@ -76,7 +82,7 @@ public class TestPriorityCompactionQueue {
} }
protected void addRegion(PriorityCompactionQueue pq, HRegion r, int p) { protected void addRegion(PriorityCompactionQueue pq, HRegion r, int p) {
pq.add(r, p); pq.add(new CompactionRequest(r, null, p));
try { try {
// Sleep 1 millisecond so 2 things are not put in the queue within the // Sleep 1 millisecond so 2 things are not put in the queue within the
// same millisecond. The queue breaks ties arbitrarily between two // same millisecond. The queue breaks ties arbitrarily between two