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/04 19:18:22 UTC

svn commit: r663314 - in /hadoop/hbase/trunk: ./ conf/ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/master/ src/java/org/apache/hadoop/hbase/regionserver/ src/java/org/apache/hadoop/hbase/util/ src/test/org/apache/hadoop/hbase/uti...

Author: stack
Date: Wed Jun  4 10:18:21 2008
New Revision: 663314

URL: http://svn.apache.org/viewvc?rev=663314&view=rev
Log:
HBASE-533 Region Historian

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/conf/hbase-env.sh
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ProcessRegionOpen.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/FSUtils.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Merge.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMetaUtils.java
    hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml
    hadoop/hbase/trunk/src/webapps/master/table.jsp

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Wed Jun  4 10:18:21 2008
@@ -47,8 +47,6 @@
                is subclassable) (David Alves via Stack)
    HBASE-603   When an exception bubbles out of getRegionServerWithRetries, wrap 
                the exception with a RetriesExhaustedException
-   HBASE-47    Option to set TTL for columns in hbase
-               (Andrew Purtell via Bryan Duxbury and Stack)
    HBASE-600   Filters have excessive DEBUG logging
    HBASE-611   regionserver should do basic health check before reporting
                alls-well to the master
@@ -56,12 +54,10 @@
    HBASE-538   Improve exceptions that come out on client-side
    HBASE-569   DemoClient.php (Jim R. Wilson via Stack)
    HBASE-522   Where new Text(string) might be used in client side method calls,
-               add an overload that takes string (Done as part of HBASE-82)
+               add an overload that takes String (Done as part of HBASE-82)
    HBASE-570   Remove HQL unit test (Done as part of HBASE-82 commit).
    HBASE-626   Use Visitor pattern in MetaRegion to reduce code clones in HTable
                and HConnectionManager (Jean-Daniel Cryans via Stack)
-   HBASE-23    UI listing regions should be sorted by address and show additional
-               region state (Jean-Daniel Cryans via Stack)
    HBASE-621   Make MAX_VERSIONS work like TTL: In scans and gets, check
                MAX_VERSIONs setting and return that many only rather than wait on
                compaction (Jean-Daniel Cryans via Stack)
@@ -75,7 +71,12 @@
    HBASE-632   HTable.getMetadata is very inefficient
 
   NEW FEATURES
+   HBASE-47    Option to set TTL for columns in hbase
+               (Andrew Purtell via Bryan Duxbury and Stack)
+   HBASE-23    UI listing regions should be sorted by address and show additional
+               region state (Jean-Daniel Cryans via Stack)
    HBASE-639   Add HBaseAdmin.getTableDescriptor function
+   HBASE-533   Region Historian
    
 Release 0.1.2 - 05/13/2008
    

Modified: hadoop/hbase/trunk/conf/hbase-env.sh
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/conf/hbase-env.sh?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/conf/hbase-env.sh (original)
+++ hadoop/hbase/trunk/conf/hbase-env.sh Wed Jun  4 10:18:21 2008
@@ -23,6 +23,7 @@
 
 # The java implementation to use.  Required.
 # export JAVA_HOME=/usr/lib/j2sdk1.5-sun
+export JAVA_HOME=/usr
 
 # Extra Java CLASSPATH elements.  Optional.
 # export HBASE_CLASSPATH=

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConstants.java Wed Jun  4 10:18:21 2008
@@ -38,8 +38,11 @@
   /** name of version file */
   static final String VERSION_FILE_NAME = "hbase.version";
   
-  /** version of file system */
-  static final String FILE_SYSTEM_VERSION = "2";
+  /**
+   * Current version of file system
+   * Version 3 added 'historian' family to .META.
+   */
+  public static final String FILE_SYSTEM_VERSION = "3";
   
   // Configuration parameters
   
@@ -129,9 +132,15 @@
   
   /** The ROOT and META column family (string) */
   static final String COLUMN_FAMILY_STR = "info:";
+  
+  /** The META historian column family (string) */
+  static final String COLUMN_FAMILY_HISTORIAN_STR = "historian:";
 
   /** The ROOT and META column family */
   static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR);
+  
+  /** The META historian column family */
+  static final byte [] COLUMN_FAMILY_HISTORIAN = Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR);
 
   /** Array of meta column names */
   static final byte[][] COLUMN_FAMILY_ARRAY = new byte[][] {COLUMN_FAMILY};
@@ -207,4 +216,5 @@
   public static final String HBASE_CLIENT_RETRIES_NUMBER_KEY =
     "hbase.client.retries.number";
   public static final int DEFAULT_CLIENT_RETRIES = 5;
+  
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTableDescriptor.java Wed Jun  4 10:18:21 2008
@@ -38,18 +38,21 @@
  */
 public class HTableDescriptor implements WritableComparable {
   /** Table descriptor for <core>-ROOT-</code> catalog table */
-  public static final HTableDescriptor ROOT_TABLEDESC =
-    new HTableDescriptor(HConstants.ROOT_TABLE_NAME,
-        new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
-            HColumnDescriptor.CompressionType.NONE, false, false,
-            Integer.MAX_VALUE, HConstants.FOREVER, null));
+  public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor(
+      HConstants.ROOT_TABLE_NAME,
+      new HColumnDescriptor[] { new HColumnDescriptor(HConstants.COLUMN_FAMILY,
+          1, HColumnDescriptor.CompressionType.NONE, false, false,
+          Integer.MAX_VALUE, HConstants.FOREVER, null) });
   
   /** Table descriptor for <code>.META.</code> catalog table */
-  public static final HTableDescriptor META_TABLEDESC =
-    new HTableDescriptor(HConstants.META_TABLE_NAME,
-        new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
-            HColumnDescriptor.CompressionType.NONE, false, false,
-            Integer.MAX_VALUE, HConstants.FOREVER, null));
+  public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
+      HConstants.META_TABLE_NAME, new HColumnDescriptor[] {
+          new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1,
+              HColumnDescriptor.CompressionType.NONE, false, false,
+              Integer.MAX_VALUE, HConstants.FOREVER, null),
+          new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN,
+              HConstants.ALL_VERSIONS, HColumnDescriptor.CompressionType.NONE,
+              false, false, Integer.MAX_VALUE, HConstants.FOREVER, null) });
   
   private boolean rootregion = false;
   private boolean metaregion = false;
@@ -64,11 +67,13 @@
    * Private constructor used internally creating table descriptors for 
    * catalog tables: e.g. .META. and -ROOT-.
    */
-  private HTableDescriptor(final byte [] name, HColumnDescriptor family) {
+  private HTableDescriptor(final byte [] name, HColumnDescriptor[] families) {
     this.name = name.clone();
     this.rootregion = Bytes.equals(name, HConstants.ROOT_TABLE_NAME);
     this.metaregion = true;
-    this.families.put(Bytes.mapKey(family.getName()), family);
+    for(HColumnDescriptor descriptor : families) {
+      this.families.put(Bytes.mapKey(descriptor.getName()), descriptor);
+    }
   }
 
   /**

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=663314&r1=663313&r2=663314&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 Wed Jun  4 10:18:21 2008
@@ -24,6 +24,7 @@
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.RegionHistorian;
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -81,6 +82,7 @@
           b.put(COL_SERVER, Bytes.toBytes(serverAddress.toString()));
           b.put(COL_STARTCODE, startCode);
           server.batchUpdate(metaRegionName, b);
+          RegionHistorian.addRegionOpen(regionInfo, serverAddress);
           if (isMetaTable) {
             // It's a meta region.
             MetaRegion m = new MetaRegion(serverAddress,

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=663314&r1=663313&r2=663314&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 Wed Jun  4 10:18:21 2008
@@ -42,6 +42,7 @@
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HServerLoad;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.RegionHistorian;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.HMsg;
@@ -258,6 +259,7 @@
           Bytes.toString(regionInfo.getRegionName())+
           " to server " + serverName);
         unassignedRegions.put(regionInfo, Long.valueOf(now));
+        RegionHistorian.addRegionAssignment(regionInfo, serverName);
         returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, regionInfo));
         if (--nregions <= 0) {
           break;
@@ -383,6 +385,7 @@
           Bytes.toString(regionInfo.getRegionName()) +
           " to the only server " + serverName);
       unassignedRegions.put(regionInfo, Long.valueOf(now));
+      RegionHistorian.addRegionAssignment(regionInfo, serverName);
       returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, regionInfo));
     }
   }

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=663314&r1=663313&r2=663314&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 Wed Jun  4 10:18:21 2008
@@ -50,6 +50,7 @@
 import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.RegionHistorian;
 import org.apache.hadoop.hbase.WrongRegionException;
 import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.io.BatchOperation;
@@ -433,6 +434,7 @@
       LOG.debug("Opening region " + this + "/" +
         this.regionInfo.getEncodedName());
     }
+    
     this.regionCompactionDir =
       new Path(getCompactionDir(basedir), encodedNameStr);
 
@@ -774,6 +776,10 @@
         LOG.debug("Cleaned up " + FSUtils.getPath(splits) + " " + deleted);
       }
       HRegion regions[] = new HRegion [] {regionA, regionB};
+      
+      RegionHistorian.addRegionSplit(this.regionInfo,
+          regionA.getRegionInfo(), regionB.getRegionInfo());
+      
       return regions;
     }
   }
@@ -865,8 +871,11 @@
         }
       }
       doRegionCompactionCleanup();
-      LOG.info("compaction completed on region " + this + " in " +
-        StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime));
+      String timeTaken = StringUtils.formatTimeDiff(System.currentTimeMillis(), 
+          startTime);
+      LOG.info("compaction completed on region " + this + " in " + timeTaken);
+      
+      RegionHistorian.addRegionCompaction(regionInfo, timeTaken);
     } finally {
       synchronized (writestate) {
         writestate.compacting = false;
@@ -1040,10 +1049,14 @@
     }
     
     if (LOG.isDebugEnabled()) {
+      String timeTaken = StringUtils.formatTimeDiff(System.currentTimeMillis(), 
+          startTime);
       LOG.debug("Finished memcache flush for region " + this +
         " in " +
           (System.currentTimeMillis() - startTime) + "ms, sequence id=" +
           sequenceId);
+      if (!regionInfo.isMetaRegion())
+        RegionHistorian.addRegionFlush(regionInfo, timeTaken);
     }
     return true;
   }
@@ -1916,6 +1929,8 @@
     Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
     FileSystem fs = FileSystem.get(conf);
     fs.mkdirs(regionDir);
+    if (!info.isMetaRegion())
+      RegionHistorian.addRegionCreation(info);
     return new HRegion(tableDir,
       new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null),
       fs, conf, info, null, null);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/FSUtils.java Wed Jun  4 10:18:21 2008
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.util;
 
 import java.io.DataInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -34,6 +35,7 @@
 import org.apache.hadoop.fs.Path;
 
 import org.apache.hadoop.hbase.FileSystemVersionException;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
@@ -193,4 +195,23 @@
     fs.delete(HStoreFile.getMapDir(tabledir, encodedRegionName, family), true);
     fs.delete(HStoreFile.getInfoDir(tabledir, encodedRegionName, family), true);
   }
+  
+  /**
+   * @param c
+   * @return Path to hbase root directory: i.e. <code>hbase.rootdir</code> as a
+   * Path.
+   * @throws IOException 
+   */
+  public static Path getRootDir(final HBaseConfiguration c) throws IOException {
+    FileSystem fs = FileSystem.get(c);
+    // Get root directory of HBase installation
+    Path rootdir = fs.makeQualified(new Path(c.get(HConstants.HBASE_DIR)));
+    if (!fs.exists(rootdir)) {
+      String message = "HBase root directory " + rootdir.toString() +
+        " does not exist.";
+      LOG.error(message);
+      throw new FileNotFoundException(message);
+    }
+    return rootdir;
+  }
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Merge.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Merge.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Merge.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Merge.java Wed Jun  4 10:18:21 2008
@@ -98,7 +98,7 @@
     // Initialize MetaUtils and and get the root of the HBase installation
     
     this.utils = new MetaUtils(conf);
-    this.rootdir = utils.initialize();
+    this.rootdir = FSUtils.getRootDir(this.conf);
     try {
       if (isMetaTable) {
         mergeTwoMetaRegions();
@@ -120,7 +120,7 @@
       return -1;
     
     } finally {
-      if (this.utils != null && this.utils.isInitialized()) {
+      if (this.utils != null) {
         this.utils.shutdown();
       }
     }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java Wed Jun  4 10:18:21 2008
@@ -29,7 +29,6 @@
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.commons.httpclient.methods.GetMethod;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
@@ -39,11 +38,9 @@
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.HTableDescriptor;
 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.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.regionserver.HLog;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -51,13 +48,13 @@
 /**
  * Contains utility methods for manipulating HBase meta tables.
  * Be sure to call {@link #shutdown()} when done with this class so it closes
- * resources opened during meta processing (ROOT, META, etc.).
+ * resources opened during meta processing (ROOT, META, etc.).  Be careful
+ * how you use this class.  If used during migrations, be careful when using
+ * this class to check whether migration is needed.
  */
 public class MetaUtils {
   private static final Log LOG = LogFactory.getLog(MetaUtils.class);
-  
   private final HBaseConfiguration conf;
-  boolean initialized;
   private FileSystem fs;
   private Path rootdir;
   private HLog log;
@@ -65,62 +62,44 @@
   private Map<byte [], HRegion> metaRegions = Collections.synchronizedSortedMap(
     new TreeMap<byte [], HRegion>(Bytes.BYTES_COMPARATOR));
   
-  /** Default constructor */
-  public MetaUtils() {
+  /** Default constructor 
+   * @throws IOException */
+  public MetaUtils() throws IOException {
     this(new HBaseConfiguration());
   }
   
-  /** @param conf HBaseConfiguration */
-  public MetaUtils(HBaseConfiguration conf) {
+  /** @param conf HBaseConfiguration 
+   * @throws IOException */
+  public MetaUtils(HBaseConfiguration conf) throws IOException {
     this.conf = conf;
     conf.setInt("hbase.client.retries.number", 1);
-    this.initialized = false;
     this.rootRegion = null;
+    initialize();
   }
 
   /**
    * Verifies that DFS is available and that HBase is off-line.
-   * 
-   * @return Path of root directory of HBase installation
    * @throws IOException
    */
-  public Path initialize() throws IOException {
-    if (!initialized) {
-      this.fs = FileSystem.get(this.conf);              // get DFS handle
-
-      // Get root directory of HBase installation
-
-      this.rootdir =
-        fs.makeQualified(new Path(this.conf.get(HConstants.HBASE_DIR)));
-
-      if (!fs.exists(rootdir)) {
-        String message = "HBase root directory " + rootdir.toString() +
+  private void initialize() throws IOException {
+    this.fs = FileSystem.get(this.conf);              // get DFS handle
+    // Get root directory of HBase installation
+    this.rootdir = fs.makeQualified(new Path(this.conf.get(HConstants.HBASE_DIR)));
+    if (!fs.exists(rootdir)) {
+      String message = "HBase root directory " + rootdir.toString() +
         " does not exist.";
-        LOG.error(message);
-        throw new FileNotFoundException(message);
-      }
-
-      this.log = new HLog(this.fs, 
-          new Path(this.fs.getHomeDirectory(),
-              HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis()
-          ),
-          this.conf, null
-      );
-
-      this.initialized = true;
+      LOG.error(message);
+      throw new FileNotFoundException(message);
     }
-    return this.rootdir;
-  }
-  
-  /** @return true if initialization completed successfully */
-  public boolean isInitialized() {
-    return this.initialized;
   }
-  
-  /** @return the HLog */
-  public HLog getLog() {
-    if (!initialized) {
-      throw new IllegalStateException("Must call initialize method first.");
+
+  /** @return the HLog 
+   * @throws IOException */
+  public synchronized HLog getLog() throws IOException {
+    if (this.log == null) {
+      Path logdir = new Path(this.fs.getHomeDirectory(),
+          HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis());
+      this.log = new HLog(this.fs, logdir, this.conf, null);
     }
     return this.log;
   }
@@ -130,9 +109,6 @@
    * @throws IOException
    */
   public HRegion getRootRegion() throws IOException {
-    if (!initialized) {
-      throw new IllegalStateException("Must call initialize method first.");
-    }
     if (this.rootRegion == null) {
       openRootRegion();
     }
@@ -147,9 +123,6 @@
    * @throws IOException
    */
   public HRegion getMetaRegion(HRegionInfo metaInfo) throws IOException {
-    if (!initialized) {
-      throw new IllegalStateException("Must call initialize method first.");
-    }
     HRegion meta = metaRegions.get(metaInfo.getRegionName());
     if (meta == null) {
       meta = openMetaRegion(metaInfo);
@@ -183,14 +156,15 @@
       metaRegions.clear();
     }
     try {
-      this.log.rollWriter();
-      this.log.closeAndDelete();
+      if (this.log != null) {
+        this.log.rollWriter();
+        this.log.closeAndDelete();
+      }
     } catch (IOException e) {
       LOG.error("closing HLog", e);
     } finally {
       this.log = null;
     }
-    this.initialized = false;
   }
 
   /**
@@ -216,10 +190,6 @@
    * @throws IOException
    */
   public void scanRootRegion(ScannerListener listener) throws IOException {
-    if (!initialized) {
-      throw new IllegalStateException("Must call initialize method first.");
-    }
-    
     // Open root region so we can scan it
     if (this.rootRegion == null) {
       openRootRegion();
@@ -265,9 +235,6 @@
   public void scanMetaRegion(HRegionInfo metaRegionInfo,
     ScannerListener listener)
   throws IOException {
-    if (!initialized) {
-      throw new IllegalStateException("Must call initialize method first.");
-    }
     // Open meta region so we can scan it
     HRegion metaRegion = openMetaRegion(metaRegionInfo);
     scanMetaRegion(metaRegion, listener);
@@ -306,15 +273,19 @@
     }
   }
 
-  private void openRootRegion() throws IOException {
+  private synchronized HRegion openRootRegion() throws IOException {
+    if (this.rootRegion != null) {
+      return this.rootRegion;
+    }
     this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO,
-        this.rootdir, this.log, this.conf);
+      this.rootdir, getLog(), this.conf);
     this.rootRegion.compactStores();
+    return this.rootRegion;
   }
 
   private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException {
     HRegion meta =
-      HRegion.openHRegion(metaInfo, this.rootdir, this.log, this.conf);
+      HRegion.openHRegion(metaInfo, this.rootdir, getLog(), this.conf);
     meta.compactStores();
     return meta;
   }
@@ -360,7 +331,7 @@
   public void addColumn(final byte [] tableName,
       final HColumnDescriptor hcd)
   throws IOException {
-    List<HRegionInfo> metas = getMETARowsInROOT();
+    List<HRegionInfo> metas = getMETARows(tableName);
     for (HRegionInfo hri: metas) {
       final HRegion m = getMetaRegion(hri);
       scanMetaRegion(m, new ScannerListener() {
@@ -393,7 +364,7 @@
    */
   public void deleteColumn(final byte [] tableName,
       final byte [] columnFamily) throws IOException {
-    List<HRegionInfo> metas = getMETARowsInROOT();
+    List<HRegionInfo> metas = getMETARows(tableName);
     final Path tabledir = new Path(rootdir, Bytes.toString(tableName));
     for (HRegionInfo hri: metas) {
       final HRegion m = getMetaRegion(hri);
@@ -440,19 +411,27 @@
   }
 
   /**
-   * @return List of <code>.META.<code> {@link HRegionInfo} found in the
-   * <code>-ROOT-</code> table.
+   * @return List of {@link HRegionInfo} rows found in the ROOT or META
+   * catalog table.
+   * @param tableName Name of table to go looking for.
    * @throws IOException
    * @see #getMetaRegion(HRegionInfo)
    */
-  public List<HRegionInfo> getMETARowsInROOT() throws IOException {
-    if (!initialized) {
-      throw new IllegalStateException("Must call initialize method first.");
-    }
+  public List<HRegionInfo> getMETARows(final byte [] tableName)
+  throws IOException {
     final List<HRegionInfo> result = new ArrayList<HRegionInfo>();
+    // If passed table name is META, then  return the root region.
+    if (Bytes.equals(HConstants.META_TABLE_NAME, tableName)) {
+      result.add(openRootRegion().getRegionInfo());
+      return result;
+    }
+    // Return all meta regions that contain the passed tablename.
     scanRootRegion(new ScannerListener() {
+      private final Log SL_LOG = LogFactory.getLog(this.getClass());
+      
       @SuppressWarnings("unused")
       public boolean processRow(HRegionInfo info) throws IOException {
+        SL_LOG.debug("Testing " + info);
         if (Bytes.equals(info.getTableDesc().getName(),
             HConstants.META_TABLE_NAME)) {
           result.add(info);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java Wed Jun  4 10:18:21 2008
@@ -21,55 +21,73 @@
 package org.apache.hadoop.hbase.util;
 
 import java.io.BufferedReader;
-import java.io.InputStreamReader;
 import java.io.IOException;
-
+import java.io.InputStreamReader;
 import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-
-import org.apache.hadoop.util.GenericOptionsParser;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MasterNotRunningException;
-
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 
 /**
- * Perform a file system upgrade to convert older file layouts to that
- * supported by HADOOP-2478, and then to the form supported by HBASE-69
+ * Perform a file system upgrade to convert older file layouts.
+ * HBase keeps a file in hdfs named hbase.version just under the hbase.rootdir.
+ * This file holds the version of the hbase data in the Filesystem.  When the
+ * software changes in a manner incompatible with the data in the Filesystem,
+ * it updates its internal version number,
+ * {@link HConstants#FILE_SYSTEM_VERSION}.  This wrapper script manages moving
+ * the filesystem across versions until there's a match with current software's
+ * version number.
+ * 
+ * <p>This wrapper script comprises a set of migration steps.  Which steps
+ * are run depends on the span between the version of the hbase data in the
+ * Filesystem and the version of the current softare.
+ * 
+ * <p>A migration script must accompany any patch that changes data formats.
+ * 
+ * <p>This script has a 'check' and 'excecute' mode.  Adding migration steps,
+ * its important to keep this in mind.  Testing if migration needs to be run,
+ * be careful not to make presumptions about the current state of the data in
+ * the filesystem.  It may have a format from many versions previous with
+ * layout not as expected or keys and values of a format not expected.  Tools
+ * such as {@link MetaUtils} may not work as expected when running against
+ * old formats -- or, worse, fail in ways that are hard to figure (One such is
+ * edits made by previous migration steps not being apparent in later migration
+ * steps).  The upshot is always verify presumptions migrating.
  */
 public class Migrate extends Configured implements Tool {
-  static final Log LOG = LogFactory.getLog(Migrate.class);
+  private static final Log LOG = LogFactory.getLog(Migrate.class);
 
   private static final String OLD_PREFIX = "hregion_";
 
   private final HBaseConfiguration conf;
-  FileSystem fs;
-  Path rootdir;
-  MetaUtils utils;
+  private FileSystem fs;
+  
+  // Gets set by migration methods if we are in readOnly mode.
+  private boolean migrationNeeded = false;
 
   /** Action to take when an extra file or unrecoverd log file is found */
   private static String ACTIONS = "abort|ignore|delete|prompt";
@@ -95,8 +113,6 @@
   }
 
   private boolean readOnly = false;
-  private boolean migrationNeeded = false;
-  private boolean newRootRegion = false;
   private ACTION otherFiles = ACTION.IGNORE;
 
   private BufferedReader reader = null;
@@ -116,14 +132,12 @@
     this.conf = conf;
   }
   
-  /** {@inheritDoc} */
-  public int run(String[] args) {
-    if (parseArgs(args) != 0) {
-      return -1;
-    }
-
+  /*
+   * Sets the hbase rootdir as fs.default.name.
+   * @return True if succeeded.
+   */
+  private boolean setFsDefaultName() {
     // Validate root directory path
-    
     Path rd = new Path(conf.get(HConstants.HBASE_DIR));
     try {
       // Validate root directory path
@@ -132,69 +146,95 @@
       LOG.fatal("Not starting migration because the root directory path '" +
           rd.toString() + "' is not valid. Check the setting of the" +
           " configuration parameter '" + HConstants.HBASE_DIR + "'", e);
-      return -1;
+      return false;
     }
     this.conf.set("fs.default.name", rd.toString());
+    return true;
+  }
 
+  /*
+   * @return True if succeeded verifying filesystem.
+   */
+  private boolean verifyFilesystem() {
     try {
       // Verify file system is up.
       fs = FileSystem.get(conf);                        // get DFS handle
       LOG.info("Verifying that file system is available...");
       FSUtils.checkFileSystemAvailable(fs);
+      return true;
     } catch (IOException e) {
       LOG.fatal("File system is not available", e);
-      return -1;
+      return false;
     }
-    
+  }
+  
+  private boolean notRunning() {
     // Verify HBase is down
     LOG.info("Verifying that HBase is not running...");
     try {
       HBaseAdmin.checkHBaseAvailable(conf);
       LOG.fatal("HBase cluster must be off-line.");
-      return -1;
+      return false;
     } catch (MasterNotRunningException e) {
-      // Expected. Ignore.
+      return true;
+    }
+  }
+  
+  /** {@inheritDoc} */
+  public int run(String[] args) {
+    if (parseArgs(args) != 0) {
+      return -1;
+    }
+    if (!setFsDefaultName()) {
+      return -2;
+    }
+    if (!verifyFilesystem()) {
+      return -3;
+    }
+    if (!notRunning()) {
+      return -4;
     }
-    
-    try {
-       
-      // Initialize MetaUtils and and get the root of the HBase installation
-      
-      this.utils = new MetaUtils(conf);
-      this.rootdir = utils.initialize();
 
+    try {
       LOG.info("Starting upgrade" + (readOnly ? " check" : ""));
 
       // See if there is a file system version file
-
-      String version = FSUtils.getVersion(fs, rootdir);
+      String version = FSUtils.getVersion(fs, FSUtils.getRootDir(this.conf));
       if (version != null && 
           version.compareTo(HConstants.FILE_SYSTEM_VERSION) == 0) {
         LOG.info("No upgrade necessary.");
         return 0;
       }
 
-      // Get contents of root directory
-      
-      FileStatus[] rootFiles = getRootDirFiles();
-
+      // Dependent on which which version hbase is at, run appropriate
+      // migrations.  Be consious that scripts can be run in readOnly -- i.e.
+      // check if migration is needed -- and then in actual migrate mode.  Be
+      // careful when writing your scripts that you do not make presumption
+      // about state of the FileSystem.  For example, in script that migrates
+      // between 2 and 3, it should not presume the layout is that of v2.  If
+      // readOnly mode, the pre-v2 scripts may not have been run yet.
       if (version == null) {
+        FileStatus[] rootFiles = getRootDirFiles();
         migrateFromNoVersion(rootFiles);
         migrateToV2(rootFiles);
+        migrateToV3();
       } else if (version.compareTo("0.1") == 0) {
-        migrateToV2(rootFiles);
+        migrateToV2(getRootDirFiles());
+        migrateToV3();
       } else if (version.compareTo("2") == 0) {
-        // Nothing to do (yet)
+        migrateToV3();
+      } else if (version.compareTo("3") == 0) {
+        // Nothing to do.
       } else {
         throw new IOException("Unrecognized version: " + version);
       }
-      
+
       if (!readOnly) {
-        // set file system version
+        // Set file system version
         LOG.info("Setting file system version.");
-        FSUtils.setVersion(fs, rootdir);
+        FSUtils.setVersion(fs, FSUtils.getRootDir(this.conf));
         LOG.info("Upgrade successful.");
-      } else if (migrationNeeded) {
+      } else if (this.migrationNeeded) {
         LOG.info("Upgrade needed.");
       }
       return 0;
@@ -202,36 +242,31 @@
       LOG.fatal("Upgrade" +  (readOnly ? " check" : "") + " failed", e);
       return -1;
       
-    } finally {
-      if (utils != null && utils.isInitialized()) {
-        utils.shutdown();
-      }
     }
   }
 
   private void migrateFromNoVersion(FileStatus[] rootFiles) throws IOException {
-    LOG.info("No file system version found. Checking to see if file system " +
-        "is at revision 0.1");
-    
-    // check to see if new root region dir exists
+    LOG.info("No file system version found. Checking to see if hbase in " +
+      "Filesystem is at revision 0.1");
 
-    checkNewRootRegionDirExists();
-    
-    // check for unrecovered region server log files
+    // Check to see if new root region dir exists
+    boolean newRootRegion = checkNewRootRegionDirExists();
+    if (this.readOnly &&  !newRootRegion) {
+      this.migrationNeeded = true;
+      return;
+    }
 
+    // Check for unrecovered region server log files
     checkForUnrecoveredLogFiles(rootFiles);
 
-    // check for "extra" files and for old upgradable regions
-
+    // Check for "extra" files and for old upgradable regions
     extraFiles(rootFiles);
 
     if (!newRootRegion) {
       // find root region
-
       String rootRegion = OLD_PREFIX +
         HRegionInfo.ROOT_REGIONINFO.getEncodedName();
-
-      if (!fs.exists(new Path(rootdir, rootRegion))) {
+      if (!fs.exists(new Path(FSUtils.getRootDir(this.conf), rootRegion))) {
         throw new IOException("Cannot find root region " + rootRegion);
       } else if (readOnly) {
         migrationNeeded = true;
@@ -240,31 +275,36 @@
         scanRootRegion();
 
         // scan for left over regions
-
         extraRegions();
       }
     }
   }
   
   private void migrateToV2(FileStatus[] rootFiles) throws IOException {
-    LOG.info("Checking to see if file system is at revision 2.");
+    LOG.info("Checking to see if hbase in Filesystem is at version 2.");
     checkForUnrecoveredLogFiles(rootFiles);
   }
 
+  private void migrateToV3() throws IOException {
+    LOG.info("Checking to see if hbase in Filesystem is at version 3.");
+    addHistorianFamilyToMeta();
+  }
+
   private FileStatus[] getRootDirFiles() throws IOException {
-    FileStatus[] stats = fs.listStatus(rootdir);
+    FileStatus[] stats = fs.listStatus(FSUtils.getRootDir(this.conf));
     if (stats == null || stats.length == 0) {
       throw new IOException("No files found under root directory " +
-          rootdir.toString());
+        FSUtils.getRootDir(this.conf).toString());
     }
     return stats;
   }
   
-  private void checkNewRootRegionDirExists() throws IOException {
-    Path rootRegionDir =
-      HRegion.getRegionDir(rootdir, HRegionInfo.ROOT_REGIONINFO);
-    newRootRegion = fs.exists(rootRegionDir);
-    migrationNeeded = !newRootRegion;
+  private boolean checkNewRootRegionDirExists() throws IOException {
+    Path rootRegionDir =  HRegion.getRegionDir(FSUtils.getRootDir(this.conf),
+      HRegionInfo.ROOT_REGIONINFO);
+    boolean newRootRegion = fs.exists(rootRegionDir);
+    this.migrationNeeded = !newRootRegion;
+    return newRootRegion;
   }
 
   private void checkForUnrecoveredLogFiles(FileStatus[] rootFiles)
@@ -289,6 +329,7 @@
   private void extraFiles(FileStatus[] stats) throws IOException {
     for (int i = 0; i < stats.length; i++) {
       String name = stats[i].getPath().getName();
+      boolean newRootRegion = checkNewRootRegionDirExists();
       if (name.startsWith(OLD_PREFIX)) {
         if (!newRootRegion) {
           // We need to migrate if the new root region directory doesn't exist
@@ -296,7 +337,6 @@
           String regionName = name.substring(OLD_PREFIX.length());
           try {
             Integer.parseInt(regionName);
-
           } catch (NumberFormatException e) {
             extraFile(otherFiles, "Old region format can not be upgraded: " +
                 name, stats[i].getPath());
@@ -341,7 +381,7 @@
   void migrateRegionDir(final byte [] tableName, String oldPath)
   throws IOException {
     // Create directory where table will live
-
+    Path rootdir = FSUtils.getRootDir(this.conf);
     Path tableDir = new Path(rootdir, Bytes.toString(tableName));
     fs.mkdirs(tableDir);
 
@@ -384,35 +424,33 @@
   }
   
   private void scanRootRegion() throws IOException {
-    utils.scanRootRegion(
-      new MetaUtils.ScannerListener() {
+    final MetaUtils utils = new MetaUtils(this.conf);
+    try {
+      utils.scanRootRegion(new MetaUtils.ScannerListener() {
         public boolean processRow(HRegionInfo info) throws IOException {
           // First move the meta region to where it should be and rename
           // subdirectories as necessary
-
-          migrateRegionDir(HConstants.META_TABLE_NAME,
-              OLD_PREFIX + info.getEncodedName());
-
-          utils.scanMetaRegion(info,
-            new MetaUtils.ScannerListener() {
-              public boolean processRow(HRegionInfo tableInfo)
-              throws IOException {
-                // Move the region to where it should be and rename
-                // subdirectories as necessary
-
-                migrateRegionDir(tableInfo.getTableDesc().getName(),
-                    OLD_PREFIX + tableInfo.getEncodedName());
-                return true;
-              }
+          migrateRegionDir(HConstants.META_TABLE_NAME, OLD_PREFIX
+              + info.getEncodedName());
+          utils.scanMetaRegion(info, new MetaUtils.ScannerListener() {
+            public boolean processRow(HRegionInfo tableInfo) throws IOException {
+              // Move the region to where it should be and rename
+              // subdirectories as necessary
+              migrateRegionDir(tableInfo.getTableDesc().getName(), OLD_PREFIX
+                  + tableInfo.getEncodedName());
+              return true;
             }
-          );
+          });
           return true;
         }
-      }
-    );
+      });
+    } finally {
+      utils.shutdown();
+    }
   }
-  
+
   private void extraRegions() throws IOException {
+    Path rootdir = FSUtils.getRootDir(this.conf);
     FileStatus[] stats = fs.listStatus(rootdir);
     if (stats == null || stats.length == 0) {
       throw new IOException("No files found under root directory " +
@@ -426,7 +464,6 @@
         if (references.contains(encodedName)) {
           message =
             "Region not in meta table but other regions reference it " + name;
-
         } else {
           message = 
             "Region not in meta table and no other regions reference it " + name;
@@ -436,6 +473,38 @@
     }
   }
   
+  private void addHistorianFamilyToMeta() throws IOException {
+    if (this.migrationNeeded) {
+      // Be careful. We cannot use MetAutils if current hbase in the
+      // Filesystem has not been migrated.
+      return;
+    }
+    boolean needed = false;
+    MetaUtils utils = new MetaUtils(this.conf);
+    try {
+      List<HRegionInfo> metas = utils.getMETARows(HConstants.META_TABLE_NAME);
+      for (HRegionInfo meta : metas) {
+        if (meta.getTableDesc().
+            getFamily(HConstants.COLUMN_FAMILY_HISTORIAN) == null) {
+          needed = true;
+          break;
+        }
+      }
+      if (needed && this.readOnly) {
+        this.migrationNeeded = true;
+      } else {
+        utils.addColumn(HConstants.META_TABLE_NAME,
+          new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN,
+            HConstants.ALL_VERSIONS, HColumnDescriptor.CompressionType.NONE,
+            false, false, Integer.MAX_VALUE, HConstants.FOREVER, null));
+        LOG.info("Historian family added to .META.");
+        // Flush out the meta edits.
+      }
+    } finally {
+      utils.shutdown();
+    }
+  }
+
   @SuppressWarnings("static-access")
   private int parseArgs(String[] args) {
     Options opts = new Options();

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMetaUtils.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMetaUtils.java?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMetaUtils.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMetaUtils.java Wed Jun  4 10:18:21 2008
@@ -40,7 +40,6 @@
     this.cluster.shutdown();
     this.cluster = null;
     MetaUtils utils = new MetaUtils(this.conf);
-    utils.initialize();
     // Add a new column to the third table, getName() + '2', and remove the old.
     final byte [] editTable = Bytes.toBytes(getName() + 2);
     final byte [] newColumn = Bytes.toBytes("newcolumn:");

Modified: hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml (original)
+++ hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml Wed Jun  4 10:18:21 2008
@@ -15,6 +15,11 @@
     </servlet>
 
     <servlet>
+        <servlet-name>org.apache.hadoop.hbase.generated.master.regionhistorian_jsp</servlet-name>
+        <servlet-class>org.apache.hadoop.hbase.generated.master.regionhistorian_jsp</servlet-class>
+    </servlet>
+
+    <servlet>
         <servlet-name>org.apache.hadoop.hbase.generated.master.table_jsp</servlet-name>
         <servlet-class>org.apache.hadoop.hbase.generated.master.table_jsp</servlet-class>
     </servlet>
@@ -25,6 +30,11 @@
     </servlet-mapping>
 
     <servlet-mapping>
+        <servlet-name>org.apache.hadoop.hbase.generated.master.regionhistorian_jsp</servlet-name>
+        <url-pattern>/regionhistorian.jsp</url-pattern>
+    </servlet-mapping>
+
+    <servlet-mapping>
         <servlet-name>org.apache.hadoop.hbase.generated.master.table_jsp</servlet-name>
         <url-pattern>/table.jsp</url-pattern>
     </servlet-mapping>

Modified: hadoop/hbase/trunk/src/webapps/master/table.jsp
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/webapps/master/table.jsp?rev=663314&r1=663313&r2=663314&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/webapps/master/table.jsp (original)
+++ hadoop/hbase/trunk/src/webapps/master/table.jsp Wed Jun  4 10:18:21 2008
@@ -45,7 +45,9 @@
     for (MetaRegion meta: onlineRegions.values()) {
       int infoPort = serverToServerInfos.get(meta.getServer().getBindAddress()+":"+meta.getServer().getPort()).getInfoPort();
       String url = "http://" + meta.getServer().getHostname() + ":" + infoPort + "/";%> 
-<tr><td><%= meta.getRegionName() %></td><td><a href="<%= url %>"><%= meta.getServer().getHostname() %>:<%= meta.getServer().getPort() %></a></td><td>-</td><td><%= meta.getStartKey() %></td><td>-</td></tr>
+<tr><td><%= Bytes.toString(meta.getRegionName()) %></td>
+    <td><a href="<%= url %>"><%= meta.getServer().getHostname() %>:<%= meta.getServer().getPort() %></a></td>
+    <td>-</td><td><%= meta.getStartKey() %></td><td>-</td></tr>
 <%  } %>
 </table>
 <%} else { %>
@@ -56,11 +58,11 @@
       if(regions != null && regions.size() > 0) { %>
 <%=     tableHeader %>
 <%      for(Map.Entry<HRegionInfo, HServerAddress> hriEntry : regions.entrySet()) { %>
-<%        System.out.println(serverToServerInfos.keySet().toArray()[0].toString());
-          System.out.println(hriEntry.getValue().getHostname()+":"+hriEntry.getValue().getPort());
-          int infoPort = serverToServerInfos.get(hriEntry.getValue().getBindAddress()+":"+hriEntry.getValue().getPort()).getInfoPort();
-          String url = "http://" + hriEntry.getValue().getHostname().toString() + ":" + infoPort + "/";  %>
-<tr><td><%= hriEntry.getKey().getRegionNameAsString()%></td><td><a href="<%= url %>"><%= hriEntry.getValue().getHostname() %>:<%= hriEntry.getValue().getPort() %></a></td>
+<%        int infoPort = serverToServerInfos.get(hriEntry.getValue().getBindAddress()+":"+hriEntry.getValue().getPort()).getInfoPort();
+          String urlRegionHistorian = "/regionhistorian.jsp?regionname="+hriEntry.getKey().getRegionNameAsString();
+          String urlRegionServer = "http://" + hriEntry.getValue().getHostname().toString() + ":" + infoPort + "/";  %>
+<tr><td><a href="<%= urlRegionHistorian %>"><%= hriEntry.getKey().getRegionNameAsString()%></a></td>
+    <td><a href="<%= urlRegionServer %>"><%= hriEntry.getValue().getHostname() %>:<%= hriEntry.getValue().getPort() %></a></td>
     <td><%= hriEntry.getKey().getEncodedName()%></td> <td><%= Bytes.toString(hriEntry.getKey().getStartKey())%></td>
     <td><%= Bytes.toString(hriEntry.getKey().getEndKey())%></td></tr>
 <%      } %>