HADOOP-2274 Excess synchronization introduced by HADOOP-2139 negatively impacts performance
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@598113 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
cbe167c981
commit
c64eb9ebdc
|
@ -31,6 +31,8 @@ Trunk (unreleased changes)
|
|||
HADOOP-2161 getRow() is orders of magnitudes slower than get(), even on rows
|
||||
with one column (Clint Morgan and Stack)
|
||||
HADOOP-2040 Hudson hangs AFTER test has finished
|
||||
HADOOP-2274 Excess synchronization introduced by HADOOP-2139 negatively
|
||||
impacts performance
|
||||
|
||||
IMPROVEMENTS
|
||||
HADOOP-2401 Add convenience put method that takes writable
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -30,13 +30,14 @@ import java.util.ConcurrentModificationException;
|
|||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Vector;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Delayed;
|
||||
import java.util.concurrent.DelayQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
@ -95,10 +96,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
protected final SortedMap<Text, HRegion> onlineRegions =
|
||||
Collections.synchronizedSortedMap(new TreeMap<Text, HRegion>());
|
||||
protected final Map<Text, HRegion> retiringRegions =
|
||||
new HashMap<Text, HRegion>();
|
||||
new ConcurrentHashMap<Text, HRegion>();
|
||||
|
||||
protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
|
||||
private final Vector<HMsg> outboundMsgs = new Vector<HMsg>();
|
||||
private final List<HMsg> outboundMsgs =
|
||||
Collections.synchronizedList(new ArrayList<HMsg>());
|
||||
|
||||
final int numRetries;
|
||||
protected final int threadWakeFrequency;
|
||||
|
@ -529,6 +531,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
/** Runs periodically to determine if the HLog should be rolled */
|
||||
class LogRoller extends Thread implements LogRollListener {
|
||||
private final Integer rollLock = new Integer(0);
|
||||
private volatile boolean rollLog;
|
||||
|
||||
/** constructor */
|
||||
|
@ -539,15 +542,23 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public synchronized void run() {
|
||||
public void run() {
|
||||
while (!stopRequested.get()) {
|
||||
try {
|
||||
this.wait(threadWakeFrequency);
|
||||
while (!rollLog && !stopRequested.get()) {
|
||||
synchronized (rollLock) {
|
||||
try {
|
||||
rollLock.wait(threadWakeFrequency);
|
||||
|
||||
} catch (InterruptedException e) {
|
||||
continue;
|
||||
} catch (InterruptedException e) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!rollLog) {
|
||||
// There's only two reasons to break out of the while loop.
|
||||
// 1. Log roll requested
|
||||
// 2. Stop requested
|
||||
// so if a log roll was not requested, continue and break out of loop
|
||||
continue;
|
||||
}
|
||||
synchronized (logRollerLock) {
|
||||
|
@ -572,9 +583,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
public synchronized void logRollRequested() {
|
||||
rollLog = true;
|
||||
this.notifyAll();
|
||||
public void logRollRequested() {
|
||||
synchronized (rollLock) {
|
||||
rollLog = true;
|
||||
rollLock.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -662,8 +675,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
synchronized(outboundMsgs) {
|
||||
outboundArray =
|
||||
this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]);
|
||||
this.outboundMsgs.clear();
|
||||
}
|
||||
this.outboundMsgs.clear();
|
||||
|
||||
try {
|
||||
this.serverInfo.setLoad(new HServerLoad(requestCount.get(),
|
||||
|
@ -1017,16 +1030,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
|
||||
/** Add to the outbound message buffer */
|
||||
private void reportOpen(HRegion region) {
|
||||
synchronized(outboundMsgs) {
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, region.getRegionInfo()));
|
||||
}
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, region.getRegionInfo()));
|
||||
}
|
||||
|
||||
/** Add to the outbound message buffer */
|
||||
private void reportClose(HRegion region) {
|
||||
synchronized(outboundMsgs) {
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_CLOSE, region.getRegionInfo()));
|
||||
}
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_CLOSE, region.getRegionInfo()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1041,11 +1050,10 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||
*/
|
||||
void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA,
|
||||
HRegionInfo newRegionB) {
|
||||
synchronized(outboundMsgs) {
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_SPLIT, oldRegion));
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, newRegionA));
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, newRegionB));
|
||||
}
|
||||
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_SPLIT, oldRegion));
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, newRegionA));
|
||||
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_OPEN, newRegionB));
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
|
Loading…
Reference in New Issue