You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2008/06/06 00:22:17 UTC

svn commit: r663764 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/master/ src/java/org/apache/hadoop/hbase/regionserver/ src/test/org/apache/hadoop/hbase/ src/webapps/master/

Author: stack
Date: Thu Jun  5 15:22:13 2008
New Revision: 663764

URL: http://svn.apache.org/viewvc?rev=663764&view=rev
Log:
HBASE-668 HBASE-533 broke build

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java
    hadoop/hbase/trunk/src/webapps/master/regionhistorian.jsp

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu Jun  5 15:22:13 2008
@@ -40,6 +40,7 @@
                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

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java Thu Jun  5 15:22:13 2008
@@ -33,15 +33,18 @@
 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
+ * <code>.META.</code> 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 @@
   } 
 
   /**
-   * 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 <code>.META.</code>.  The HTable instance is lazily instantiated to 
+   * allow for the getting and storing aside of an Historian instance even
+   * in the case where <code>.META.</code> has not yet deployed.
    * @return The region historian
    */
   public static RegionHistorian getInstance() {
@@ -98,15 +98,16 @@
   /**
    * 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<RegionHistoryInformation> getRegionHistory(
-      String regionName) {
-    getInstance();
-    List<RegionHistoryInformation> informations = new ArrayList<RegionHistoryInformation>();
+  public List<RegionHistoryInformation> getRegionHistory(String regionName) {
+    if (!isOnline()) {
+      return null;
+    }
+    List<RegionHistoryInformation> informations =
+      new ArrayList<RegionHistoryInformation>();
     try {
       /*
        * TODO REGION_HISTORIAN_KEYS is used because there is no other for the
@@ -115,7 +116,7 @@
        */
       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 @@
   
   /**
    * 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 @@
    * @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 @@
 
   /**
    * 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 @@
 
   /**
    * 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 @@
    * @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 @@
    * @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 @@
     }
 
     /**
-     * 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

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/BaseScanner.java Thu Jun  5 15:22:13 2008
@@ -407,7 +407,8 @@
           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();
             }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java Thu Jun  5 15:22:13 2008
@@ -48,6 +48,7 @@
 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 @@
   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 @@
       }
 
       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 @@
       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 @@
     /*
      * Clean up and close up shop
      */
+    RegionHistorian.getInstance().offline();
     if (this.infoServer != null) {
       LOG.info("Stopping infoServer");
       try {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java Thu Jun  5 15:22:13 2008
@@ -62,6 +62,8 @@
   protected boolean process() throws IOException {
     Boolean result =
       new RetryableMetaOperation<Boolean>(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 @@
           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 @@
     }.doWithRetries();
     return result == null ? true : result;
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessServerShutdown.java Thu Jun  5 15:22:13 2008
@@ -242,7 +242,8 @@
           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();
         }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java Thu Jun  5 15:22:13 2008
@@ -118,12 +118,14 @@
   // 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 @@
           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 @@
           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 @@
       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();

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java Thu Jun  5 15:22:13 2008
@@ -80,7 +80,8 @@
   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));
   }
   
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Jun  5 15:22:13 2008
@@ -109,6 +109,7 @@
   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 @@
     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 @@
       }
       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 @@
           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 @@
         " 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 @@
    * @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 @@
    * @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);
     }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Jun  5 15:22:13 2008
@@ -63,6 +63,7 @@
 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 @@
       LOG.fatal("Unhandled exception. Aborting...", t);
       abort();
     }
+    RegionHistorian.getInstance().offline();
     this.leases.closeAfterLeasesExpire();
     this.worker.stop();
     this.server.stop();
@@ -846,9 +848,14 @@
   }
   
   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()),

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGlobalMemcacheLimit.java Thu Jun  5 15:22:13 2008
@@ -78,9 +78,9 @@
     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 @@
     
     // 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 @@
     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);

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java Thu Jun  5 15:22:13 2008
@@ -165,14 +165,11 @@
         + " 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) {

Modified: hadoop/hbase/trunk/src/webapps/master/regionhistorian.jsp
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/webapps/master/regionhistorian.jsp?rev=663764&r1=663763&r2=663764&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/webapps/master/regionhistorian.jsp (original)
+++ hadoop/hbase/trunk/src/webapps/master/regionhistorian.jsp Thu Jun  5 15:22:13 2008
@@ -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<RegionHistoryInformation> informations = RegionHistorian.getRegionHistory(regionName);
+  HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
+  List<RegionHistoryInformation> informations = RegionHistorian.getInstance().getRegionHistory(regionName);
 %><?xml version="1.0" encoding="UTF-8" ?>
 <!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" 
   "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">