diff --git a/CHANGES.txt b/CHANGES.txt
index 8ae4a14db10..f56801f31c6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -40,6 +40,7 @@ Hbase Change Log
to enable/disable table
HBASE-654 API HTable.getMetadata().addFamily shouldn't be exposed to user
HBASE-666 UnmodifyableHRegionInfo gives the wrong encoded name
+ HBASE-668 HBASE-533 broke build
IMPROVEMENTS
HBASE-559 MR example job to count table rows
diff --git a/src/java/org/apache/hadoop/hbase/RegionHistorian.java b/src/java/org/apache/hadoop/hbase/RegionHistorian.java
index b5022ddbae3..dd3520a36f4 100644
--- a/src/java/org/apache/hadoop/hbase/RegionHistorian.java
+++ b/src/java/org/apache/hadoop/hbase/RegionHistorian.java
@@ -33,15 +33,18 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.util.Bytes;
+
/**
* The Region Historian task is to keep track of every modification a region
- * has to go trought. Public methods are used to update the information in the
- * .META. table and to retreive it.
+ * has to go through. Public methods are used to update the information in the
+ * .META.
table and to retrieve it. This is a Singleton. By
+ * default, the Historian is offline; it will not log. Its enabled in the
+ * regionserver and master down in their guts after there's some certainty the
+ * .META. has been deployed.
*/
public class RegionHistorian implements HConstants {
-
- static final Log LOG = LogFactory.getLog(RegionHistorian.class);
-
+ private static final Log LOG = LogFactory.getLog(RegionHistorian.class);
+
private HTable metaTable;
private GregorianCalendar cal = new GregorianCalendar();
@@ -69,23 +72,20 @@ public class RegionHistorian implements HConstants {
}
/**
- * Default constructor. Initializes reference to .META. table
- *
+ * Default constructor. Initializes reference to .META. table. Inaccessible.
+ * Use {@link #getInstance(HBaseConfiguration)} to obtain the Singleton
+ * instance of this class.
*/
private RegionHistorian() {
- HBaseConfiguration conf = new HBaseConfiguration();
-
- try {
- metaTable = new HTable(conf, META_TABLE_NAME);
- LOG.debug("Region historian is ready.");
- } catch (IOException ioe) {
- LOG.warn("Unable to create RegionHistorian", ioe);
- }
+ super();
}
/**
- * Singleton method
- *
+ * Get the RegionHistorian Singleton instance.
+ * @param c Configuration to use. Used to create an {@link HTable} homed
+ * on .META.
. The HTable instance is lazily instantiated to
+ * allow for the getting and storing aside of an Historian instance even
+ * in the case where .META.
has not yet deployed.
* @return The region historian
*/
public static RegionHistorian getInstance() {
@@ -98,15 +98,16 @@ public class RegionHistorian implements HConstants {
/**
* Returns, for a given region name, an ordered list by timestamp of all
* values in the historian column of the .META. table.
- *
* @param regionName
* Region name as a string
- * @return List of RegionHistoryInformation
+ * @return List of RegionHistoryInformation or null if we're offline.
*/
- public static List getRegionHistory(
- String regionName) {
- getInstance();
- List informations = new ArrayList();
+ public List getRegionHistory(String regionName) {
+ if (!isOnline()) {
+ return null;
+ }
+ List informations =
+ new ArrayList();
try {
/*
* TODO REGION_HISTORIAN_KEYS is used because there is no other for the
@@ -115,7 +116,7 @@ public class RegionHistorian implements HConstants {
*/
for (HistorianColumnKey keyEnu : HistorianColumnKey.values()) {
byte[] columnKey = keyEnu.key;
- Cell[] cells = historian.metaTable.get(Bytes.toBytes(regionName),
+ Cell[] cells = this.metaTable.get(Bytes.toBytes(regionName),
columnKey, ALL_VERSIONS);
if (cells != null) {
for (Cell cell : cells) {
@@ -134,33 +135,27 @@ public class RegionHistorian implements HConstants {
/**
* Method to add a creation event to the row in the .META table
- *
* @param info
*/
- public static void addRegionAssignment(HRegionInfo info, String serverName) {
-
+ public void addRegionAssignment(HRegionInfo info, String serverName) {
add(HistorianColumnKey.REGION_ASSIGNMENT.key, "Region assigned to server "
+ serverName, info);
}
/**
* Method to add a creation event to the row in the .META table
- *
* @param info
*/
- public static void addRegionCreation(HRegionInfo info) {
-
+ public void addRegionCreation(HRegionInfo info) {
add(HistorianColumnKey.REGION_CREATION.key, "Region creation", info);
}
/**
* Method to add a opening event to the row in the .META table
- *
* @param info
* @param address
*/
- public static void addRegionOpen(HRegionInfo info, HServerAddress address) {
-
+ public void addRegionOpen(HRegionInfo info, HServerAddress address) {
add(HistorianColumnKey.REGION_OPEN.key, "Region opened on server : "
+ address.getHostname(), info);
}
@@ -172,9 +167,8 @@ public class RegionHistorian implements HConstants {
* @param newInfo1
* @param newInfo2
*/
- public static void addRegionSplit(HRegionInfo oldInfo, HRegionInfo newInfo1,
- HRegionInfo newInfo2) {
-
+ public void addRegionSplit(HRegionInfo oldInfo, HRegionInfo newInfo1,
+ HRegionInfo newInfo2) {
HRegionInfo[] infos = new HRegionInfo[] { newInfo1, newInfo2 };
for (HRegionInfo info : infos) {
add(HistorianColumnKey.REGION_SPLIT.key, "Region split from : "
@@ -184,10 +178,9 @@ public class RegionHistorian implements HConstants {
/**
* Method to add a compaction event to the row in the .META table
- *
* @param info
*/
- public static void addRegionCompaction(HRegionInfo info, String timeTaken) {
+ public void addRegionCompaction(HRegionInfo info, String timeTaken) {
if (LOG.isDebugEnabled()) {
add(HistorianColumnKey.REGION_COMPACTION.key,
"Region compaction completed in " + timeTaken, info);
@@ -196,10 +189,9 @@ public class RegionHistorian implements HConstants {
/**
* Method to add a flush event to the row in the .META table
- *
* @param info
*/
- public static void addRegionFlush(HRegionInfo info, String timeTaken) {
+ public void addRegionFlush(HRegionInfo info, String timeTaken) {
if (LOG.isDebugEnabled()) {
add(HistorianColumnKey.REGION_FLUSH.key, "Region flush completed in "
+ timeTaken, info);
@@ -212,7 +204,8 @@ public class RegionHistorian implements HConstants {
* @param text
* @param info
*/
- private static void add(byte[] column, String text, HRegionInfo info) {
+ private void add(byte[] column,
+ String text, HRegionInfo info) {
add(column, text, info, LATEST_TIMESTAMP);
}
@@ -223,14 +216,18 @@ public class RegionHistorian implements HConstants {
* @param info
* @param timestamp
*/
- private static void add(byte[] column, String text, HRegionInfo info, long timestamp) {
+ private void add(byte[] column,
+ String text, HRegionInfo info, long timestamp) {
+ if (!isOnline()) {
+ // Its a noop
+ return;
+ }
if (!info.isMetaRegion()) {
- getInstance();
BatchUpdate batch = new BatchUpdate(info.getRegionName());
batch.setTimestamp(timestamp);
batch.put(column, Bytes.toBytes(text));
try {
- historian.metaTable.commit(batch);
+ this.metaTable.commit(batch);
} catch (IOException ioe) {
LOG.warn("Unable to '" + text + "'", ioe);
}
@@ -277,15 +274,38 @@ public class RegionHistorian implements HConstants {
}
/**
- * Returns the value of the timestamp processed
- * with the date formater.
- * @return
+ * @return The value of the timestamp processed with the date formater.
*/
public String getTimestampAsString() {
cal.setTimeInMillis(timestamp);
return dateFormat.format(cal.getTime());
}
-
}
-}
+ /**
+ * @return True if the historian is online. When offline, will not add
+ * updates to the .META. table.
+ */
+ public boolean isOnline() {
+ return this.metaTable != null;
+ }
+
+ /**
+ * @param c Online the historian. Invoke after cluster has spun up.
+ */
+ public void online(final HBaseConfiguration c) {
+ try {
+ this.metaTable = new HTable(c, META_TABLE_NAME);
+ } catch (IOException ioe) {
+ LOG.error("Unable to create RegionHistorian", ioe);
+ }
+ }
+
+ /**
+ * Offlines the historian.
+ * @see #online(HBaseConfiguration)
+ */
+ public void offline() {
+ this.metaTable = null;
+ }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/hbase/master/BaseScanner.java b/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
index 3ef9898427a..7d0a8127e2f 100644
--- a/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
+++ b/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
@@ -407,7 +407,8 @@ abstract class BaseScanner extends Chore implements HConstants {
if (master.fs.exists(logDir)) {
regionManager.splitLogLock.lock();
try {
- HLog.splitLog(master.rootdir, logDir, master.fs, master.conf);
+ HLog.splitLog(master.rootdir, logDir, master.fs,
+ master.getConfiguration());
} finally {
regionManager.splitLogLock.unlock();
}
diff --git a/src/java/org/apache/hadoop/hbase/master/HMaster.java b/src/java/org/apache/hadoop/hbase/master/HMaster.java
index 319e21f450b..234c2b65217 100644
--- a/src/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/src/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.RegionHistorian;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -100,7 +101,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
volatile boolean shutdownRequested = false;
volatile boolean fsOk = true;
final Path rootdir;
- final HBaseConfiguration conf;
+ private final HBaseConfiguration conf;
final FileSystem fs;
final Random rand;
final int threadWakeFrequency;
@@ -204,24 +205,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
}
if (!fs.exists(rootRegionDir)) {
- LOG.info("BOOTSTRAP: creating ROOT and first META regions");
- try {
- HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO,
- this.rootdir, this.conf);
- HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO,
- this.rootdir, this.conf);
-
- // Add first region from the META table to the ROOT region.
- HRegion.addRegionToMETA(root, meta);
- root.close();
- root.getLog().closeAndDelete();
- meta.close();
- meta.getLog().closeAndDelete();
- } catch (IOException e) {
- e = RemoteExceptionHandler.checkIOException(e);
- LOG.error("bootstrap", e);
- throw e;
- }
+ bootstrap();
}
} catch (IOException e) {
LOG.fatal("Not starting HMaster because:", e);
@@ -247,15 +231,36 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
conf.getInt("hbase.master.meta.thread.rescanfrequency", 60 * 1000);
this.sleeper = new Sleeper(this.threadWakeFrequency, this.closed);
-
+
serverManager = new ServerManager(this);
regionManager = new RegionManager(this);
-
+
// We're almost open for business
this.closed.set(false);
LOG.info("HMaster initialized on " + this.address.toString());
}
+ private void bootstrap() throws IOException {
+ LOG.info("BOOTSTRAP: creating ROOT and first META regions");
+ try {
+ HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO,
+ this.rootdir, this.conf);
+ HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO,
+ this.rootdir, this.conf);
+
+ // Add first region from the META table to the ROOT region.
+ HRegion.addRegionToMETA(root, meta);
+ root.close();
+ root.getLog().closeAndDelete();
+ meta.close();
+ meta.getLog().closeAndDelete();
+ } catch (IOException e) {
+ e = RemoteExceptionHandler.checkIOException(e);
+ LOG.error("bootstrap", e);
+ throw e;
+ }
+ }
+
/**
* Checks to see if the file system is still accessible.
* If not, sets closed
@@ -363,6 +368,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
/*
* Clean up and close up shop
*/
+ RegionHistorian.getInstance().offline();
if (this.infoServer != null) {
LOG.info("Stopping infoServer");
try {
diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java b/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
index a59a7f11697..cb889aac404 100644
--- a/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
+++ b/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
@@ -62,6 +62,8 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
protected boolean process() throws IOException {
Boolean result =
new RetryableMetaOperation(this.metaRegion, this.master) {
+ private final RegionHistorian historian = RegionHistorian.getInstance();
+
public Boolean call() throws IOException {
LOG.info(regionInfo.getRegionNameAsString() + " open on " +
serverAddress.toString());
@@ -82,7 +84,13 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
b.put(COL_SERVER, Bytes.toBytes(serverAddress.toString()));
b.put(COL_STARTCODE, startCode);
server.batchUpdate(metaRegionName, b);
- RegionHistorian.addRegionOpen(regionInfo, serverAddress);
+ if (!this.historian.isOnline()) {
+ // This is safest place to do the onlining of the historian in
+ // the master. When we get to here, we know there is a .META.
+ // for the historian to go against.
+ this.historian.online(this.master.getConfiguration());
+ }
+ this.historian.addRegionOpen(regionInfo, serverAddress);
if (isMetaTable) {
// It's a meta region.
MetaRegion m = new MetaRegion(serverAddress,
@@ -109,4 +117,4 @@ class ProcessRegionOpen extends ProcessRegionStatusChange {
}.doWithRetries();
return result == null ? true : result;
}
-}
+}
\ No newline at end of file
diff --git a/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java b/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
index e19a8afe089..48186d9d113 100644
--- a/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
+++ b/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
@@ -242,7 +242,8 @@ class ProcessServerShutdown extends RegionServerOperation {
return false;
}
try {
- HLog.splitLog(master.rootdir, oldLogDir, master.fs, master.conf);
+ HLog.splitLog(master.rootdir, oldLogDir, master.fs,
+ master.getConfiguration());
} finally {
master.regionManager.splitLogLock.unlock();
}
diff --git a/src/java/org/apache/hadoop/hbase/master/RegionManager.java b/src/java/org/apache/hadoop/hbase/master/RegionManager.java
index 123d5a76853..62d98c5c930 100644
--- a/src/java/org/apache/hadoop/hbase/master/RegionManager.java
+++ b/src/java/org/apache/hadoop/hbase/master/RegionManager.java
@@ -118,12 +118,14 @@ class RegionManager implements HConstants {
// How many regions to assign a server at a time.
private final int maxAssignInOneGo;
- private final HMaster master;
+ private final HMaster master;
+
+ private final RegionHistorian historian;
RegionManager(HMaster master) {
this.master = master;
-
- this.maxAssignInOneGo = this.master.conf.
+ this.historian = RegionHistorian.getInstance();
+ this.maxAssignInOneGo = this.master.getConfiguration().
getInt("hbase.regions.percheckin", 10);
// The root region
@@ -259,7 +261,7 @@ class RegionManager implements HConstants {
Bytes.toString(regionInfo.getRegionName())+
" to server " + serverName);
unassignedRegions.put(regionInfo, Long.valueOf(now));
- RegionHistorian.addRegionAssignment(regionInfo, serverName);
+ this.historian.addRegionAssignment(regionInfo, serverName);
returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, regionInfo));
if (--nregions <= 0) {
break;
@@ -385,7 +387,7 @@ class RegionManager implements HConstants {
Bytes.toString(regionInfo.getRegionName()) +
" to the only server " + serverName);
unassignedRegions.put(regionInfo, Long.valueOf(now));
- RegionHistorian.addRegionAssignment(regionInfo, serverName);
+ this.historian.addRegionAssignment(regionInfo, serverName);
returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, regionInfo));
}
}
@@ -544,8 +546,8 @@ class RegionManager implements HConstants {
byte [] metaRegionName)
throws IOException {
// 2. Create the HRegion
- HRegion region =
- HRegion.createHRegion(newRegion, master.rootdir, master.conf);
+ HRegion region = HRegion.createHRegion(newRegion, master.rootdir,
+ master.getConfiguration());
// 3. Insert into meta
HRegionInfo info = region.getRegionInfo();
diff --git a/src/java/org/apache/hadoop/hbase/master/ServerManager.java b/src/java/org/apache/hadoop/hbase/master/ServerManager.java
index 86fafd50ec3..c9feac6ab58 100644
--- a/src/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/src/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -80,7 +80,8 @@ class ServerManager implements HConstants {
public ServerManager(HMaster master) {
this.master = master;
serverLeases = new Leases(master.leaseTimeout,
- master.conf.getInt("hbase.master.lease.thread.wakefrequency", 15 * 1000));
+ master.getConfiguration().getInt("hbase.master.lease.thread.wakefrequency",
+ 15 * 1000));
}
/**
diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index a41fe1dea55..231c6b65978 100644
--- a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -109,6 +109,7 @@ public class HRegion implements HConstants {
static final Random rand = new Random();
static final Log LOG = LogFactory.getLog(HRegion.class);
final AtomicBoolean closed = new AtomicBoolean(false);
+ private final RegionHistorian historian;
/**
* Merge two HRegions. The regions must be adjacent andmust not overlap.
@@ -429,6 +430,7 @@ public class HRegion implements HConstants {
String encodedNameStr = Integer.toString(this.regionInfo.getEncodedName());
this.regiondir = new Path(basedir, encodedNameStr);
Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME);
+ this.historian = RegionHistorian.getInstance();
if (LOG.isDebugEnabled()) {
LOG.debug("Opening region " + this + "/" +
@@ -777,8 +779,8 @@ public class HRegion implements HConstants {
}
HRegion regions[] = new HRegion [] {regionA, regionB};
- RegionHistorian.addRegionSplit(this.regionInfo,
- regionA.getRegionInfo(), regionB.getRegionInfo());
+ this.historian.addRegionSplit(this.regionInfo,
+ regionA.getRegionInfo(), regionB.getRegionInfo());
return regions;
}
@@ -875,7 +877,7 @@ public class HRegion implements HConstants {
startTime);
LOG.info("compaction completed on region " + this + " in " + timeTaken);
- RegionHistorian.addRegionCompaction(regionInfo, timeTaken);
+ this.historian.addRegionCompaction(regionInfo, timeTaken);
} finally {
synchronized (writestate) {
writestate.compacting = false;
@@ -1055,8 +1057,9 @@ public class HRegion implements HConstants {
" in " +
(System.currentTimeMillis() - startTime) + "ms, sequence id=" +
sequenceId);
- if (!regionInfo.isMetaRegion())
- RegionHistorian.addRegionFlush(regionInfo, timeTaken);
+ if (!regionInfo.isMetaRegion()) {
+ this.historian.addRegionFlush(regionInfo, timeTaken);
+ }
}
return true;
}
@@ -1923,14 +1926,17 @@ public class HRegion implements HConstants {
* @throws IOException
*/
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
- final HBaseConfiguration conf) throws IOException {
+ final HBaseConfiguration conf)
+ throws IOException {
Path tableDir =
HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName());
Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
FileSystem fs = FileSystem.get(conf);
fs.mkdirs(regionDir);
- if (!info.isMetaRegion())
- RegionHistorian.addRegionCreation(info);
+ // Note in historian the creation of new region.
+ if (!info.isMetaRegion()) {
+ RegionHistorian.getInstance().addRegionCreation(info);
+ }
return new HRegion(tableDir,
new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null),
fs, conf, info, null, null);
@@ -1950,7 +1956,8 @@ public class HRegion implements HConstants {
* @throws IOException
*/
public static HRegion openHRegion(final HRegionInfo info, final Path rootDir,
- final HLog log, final HBaseConfiguration conf) throws IOException {
+ final HLog log, final HBaseConfiguration conf)
+ throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Opening region: " + info);
}
diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 81127fc9f40..f76ce714d28 100644
--- a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.LeaseListener;
import org.apache.hadoop.hbase.Leases;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.RegionHistorian;
import org.apache.hadoop.hbase.RegionServerRunningException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.UnknownScannerException;
@@ -403,6 +404,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
LOG.fatal("Unhandled exception. Aborting...", t);
abort();
}
+ RegionHistorian.getInstance().offline();
this.leases.closeAfterLeasesExpire();
this.worker.stop();
this.server.stop();
@@ -846,9 +848,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
void openRegion(final HRegionInfo regionInfo) {
+ // If historian is not online and this is not a meta region, online it.
+ if (!regionInfo.isMetaRegion() &&
+ !RegionHistorian.getInstance().isOnline()) {
+ RegionHistorian.getInstance().online(this.conf);
+ }
Integer mapKey = Bytes.mapKey(regionInfo.getRegionName());
HRegion region = this.onlineRegions.get(mapKey);
- if(region == null) {
+ if (region == null) {
try {
region = new HRegion(HTableDescriptor.getTableDir(rootDir,
regionInfo.getTableDesc().getName()),
diff --git a/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java b/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java
index 6edb863a722..5e1fb44b247 100644
--- a/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java
+++ b/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java
@@ -78,9 +78,9 @@ public class TestGlobalMemcacheLimit extends HBaseClusterTestCase {
for (HRegion region : server.getOnlineRegions()) {
region.flushcache();
}
- // make sure we're starting at 0 so that it's easy to predict what the
- // results of our tests should be.
- assertEquals("Starting memcache size", 0, server.getGlobalMemcacheSize());
+ // We used to assert that the memsize here was zero but with the addition
+ // of region historian, its no longer true; an entry is added for the
+ // flushes run above.
}
/**
@@ -93,8 +93,11 @@ public class TestGlobalMemcacheLimit extends HBaseClusterTestCase {
// make sure the region server says it is using as much memory as we think
// it is.
- assertEquals("Global memcache size", dataSize,
- server.getGlobalMemcacheSize());
+ // Global cache size is now polluted by region historian data. We used
+ // to be able to do direct compare of global memcache and the data added
+ // but not since HBASE-533 went in. Compare has to be a bit sloppy.
+ assertTrue("Global memcache size",
+ dataSize <= server.getGlobalMemcacheSize());
}
/**
@@ -115,8 +118,11 @@ public class TestGlobalMemcacheLimit extends HBaseClusterTestCase {
int preFlushRows = (int)Math.floor(numRows);
long dataAdded = populate(table1, preFlushRows, 500);
- assertEquals("Expected memcache size", dataAdded + startingDataSize,
- server.getGlobalMemcacheSize());
+ // Global cache size is now polluted by region historian data. We used
+ // to be able to do direct compare of global memcache and the data added
+ // but not since HBASE-533 went in.
+ long cacheSize = server.getGlobalMemcacheSize();
+ assertTrue("Expected memcache size", (dataAdded + startingDataSize) <= cacheSize);
populate(table1, 2, preFlushRows + 500);
assertTrue("Post-flush memcache size", server.getGlobalMemcacheSize() <= 1024 * 1024);
diff --git a/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java b/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java
index bffe45b0051..e8e982ab7b7 100644
--- a/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java
+++ b/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java
@@ -165,14 +165,11 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
+ " regions. Load Average: " + avg);
for (HRegionServer server : servers) {
- LOG.debug(server.hashCode() + " Avg: " + avg + " actual: "
- + server.getOnlineRegions().size());
-
int serverLoad = server.getOnlineRegions().size();
+ LOG.debug(server.hashCode() + " Avg: " + avg + " actual: " + serverLoad);
if (!(serverLoad <= avg + 2 && serverLoad >= avg - 2)) {
success = false;
}
-
}
if (!success) {
diff --git a/src/webapps/master/regionhistorian.jsp b/src/webapps/master/regionhistorian.jsp
index ddf0f12be66..7f23bfbf21d 100644
--- a/src/webapps/master/regionhistorian.jsp
+++ b/src/webapps/master/regionhistorian.jsp
@@ -1,10 +1,12 @@
<%@ page contentType="text/html;charset=UTF-8"
import="java.util.List"
import="org.apache.hadoop.hbase.RegionHistorian"
+ import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.RegionHistorian.RegionHistoryInformation"
import="org.apache.hadoop.hbase.HConstants"%><%
String regionName = request.getParameter("regionname");
- List informations = RegionHistorian.getRegionHistory(regionName);
+ HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
+ List informations = RegionHistorian.getInstance().getRegionHistory(regionName);
%>