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); %>