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/01 06:42:12 UTC

svn commit: r662146 - in /hadoop/hbase/trunk: ./ bin/ conf/ src/java/org/apache/hadoop/hbase/master/ src/java/org/apache/hadoop/hbase/regionserver/ src/java/org/apache/hadoop/hbase/util/ src/webapps/master/WEB-INF/

Author: stack
Date: Sat May 31 21:42:11 2008
New Revision: 662146

URL: http://svn.apache.org/viewvc?rev=662146&view=rev
Log:
HBASE-659 HLog#cacheFlushLock not cleared; hangs a region

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/bin/hirb.rb
    hadoop/hbase/trunk/conf/hbase-env.sh
    hadoop/hbase/trunk/conf/hbase-site.xml
    hadoop/hbase/trunk/conf/log4j.properties
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableDelete.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableOperation.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/MetaUtils.java
    hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Sat May 31 21:42:11 2008
@@ -34,6 +34,7 @@
    HBASE-649   API polluted with default and protected access data members and methods
    HBASE-650   Add String versions of get, scanner, put in HTable
    HBASE-656   Do not retry exceptions such as unknown scanner or illegal argument
+   HBASE-659   HLog#cacheFlushLock not cleared; hangs a region
 
   IMPROVEMENTS
    HBASE-559   MR example job to count table rows

Modified: hadoop/hbase/trunk/bin/hirb.rb
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/bin/hirb.rb?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/bin/hirb.rb (original)
+++ hadoop/hbase/trunk/bin/hirb.rb Sat May 31 21:42:11 2008
@@ -11,9 +11,9 @@
 
 # Run the java magic include and import basic HBase types.
 include Java
-import org.apache.hadoop.hbase.HBaseConfiguration
 import org.apache.hadoop.hbase.client.HTable
 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.HTableDescriptor
@@ -30,14 +30,17 @@
   puts ' version   Output HBase version'
 end
 
-def version
-  puts "Version: #{org.apache.hadoop.hbase.util.VersionInfo.getVersion()},\
+def versionstr
+  "Version: #{org.apache.hadoop.hbase.util.VersionInfo.getVersion()},\
  r#{org.apache.hadoop.hbase.util.VersionInfo.getRevision()},\
  #{org.apache.hadoop.hbase.util.VersionInfo.getDate()}"
+end 
+
+def version
+  puts versionstr()
 end
 
 # Output a banner message that tells users where to go for help
 # TODO: Test that we're in irb context.  For now presume it.
-# TODO: Test that we are in shell context.
-puts "HBase Shell; type 'hbase<RETURN>' for the list of supported HBase commands"
-version
+puts "HBase Shell; " + versionstr()
+puts "+ Type 'hbase<RETURN>' for list of HBase commands"

Modified: hadoop/hbase/trunk/conf/hbase-env.sh
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/conf/hbase-env.sh?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/conf/hbase-env.sh (original)
+++ hadoop/hbase/trunk/conf/hbase-env.sh Sat May 31 21:42:11 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/conf/hbase-site.xml
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/conf/hbase-site.xml?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/conf/hbase-site.xml (original)
+++ hadoop/hbase/trunk/conf/hbase-site.xml Sat May 31 21:42:11 2008
@@ -22,4 +22,29 @@
  */
 -->
 <configuration>
+  <property>
+    <name>hbase.master</name>
+    <value>durruti.local:60000</value>
+    <description>The host and port that the HBase master runs at.
+    A value of 'local' runs the master and a regionserver in
+    a single process.
+    </description>
+  </property>
+  <property>
+    <name>hbase.rootdir</name>
+    <value>hdfs://durruti.local:10000/hbase</value>
+    <description>The directory shared by region servers.
+    Should be fully-qualified to include the filesystem to use.
+    E.g: hdfs://NAMENODE_SERVER:PORT/HBASE_ROOTDIR
+    </description>
+  </property>
+
+<property>
+  <name>dfs.replication</name>
+  <value>1</value>
+  <description>Default block replication. 
+  The actual number of replications can be specified when the file is created.
+  The default is used if replication is not specified in create time.
+  </description>
+</property>
 </configuration>

Modified: hadoop/hbase/trunk/conf/log4j.properties
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/conf/log4j.properties?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/conf/log4j.properties (original)
+++ hadoop/hbase/trunk/conf/log4j.properties Sat May 31 21:42:11 2008
@@ -41,3 +41,6 @@
 # Custom Logging levels
 
 #log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+log4j.logger.org.apache.hadoop.fs=DEBUG
+log4j.logger.org.apache.hadoop.dfs=DEBUG
+log4j.logger.org.apache.hadoop.hbase=DEBUG

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ChangeTableState.java Sat May 31 21:42:11 2008
@@ -25,6 +25,8 @@
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.io.BatchUpdate;
@@ -33,6 +35,7 @@
 
 /** Instantiated to enable or disable a table */
 class ChangeTableState extends TableOperation {
+  private final Log LOG = LogFactory.getLog(this.getClass());
   private boolean online;
 
   protected final Map<String, HashSet<HRegionInfo>> servedRegions =

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ColumnOperation.java Sat May 31 21:42:11 2008
@@ -20,6 +20,9 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.io.BatchUpdate;
@@ -27,6 +30,7 @@
 import org.apache.hadoop.hbase.util.Writables;
 
 abstract class ColumnOperation extends TableOperation {
+  private final Log LOG = LogFactory.getLog(this.getClass());
   
   protected ColumnOperation(final HMaster master, final byte [] tableName) 
   throws IOException {
@@ -53,4 +57,4 @@
       LOG.debug("updated columns in row: " + i.getRegionName());
     }
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableDelete.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableDelete.java?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableDelete.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableDelete.java Sat May 31 21:42:11 2008
@@ -21,6 +21,8 @@
 
 import java.io.IOException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
@@ -33,6 +35,7 @@
  * Instantiated to delete a table. Table must be offline.
  */
 class TableDelete extends TableOperation {
+  private final Log LOG = LogFactory.getLog(this.getClass());
 
   TableDelete(final HMaster master, final byte [] tableName) throws IOException {
     super(master, tableName);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableOperation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableOperation.java?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableOperation.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableOperation.java Sat May 31 21:42:11 2008
@@ -25,59 +25,42 @@
 import java.util.List;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.io.RowResult;
-import org.apache.hadoop.hbase.util.Sleeper;
+import org.apache.hadoop.hbase.ipc.HRegionInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
 
 /**
  * Abstract base class for operations that need to examine all HRegionInfo 
- * objects that make up a table. (For a table, operate on each of its rows
- * in .META.) To gain the 
+ * objects in a table. (For a table, operate on each of its rows
+ * in .META.).
  */
 abstract class TableOperation implements HConstants {
-  static final Long ZERO_L = Long.valueOf(0L);
-  
-  protected static final Log LOG = LogFactory.getLog(TableOperation.class);
-  
-  protected Set<MetaRegion> metaRegions;
-  protected byte [] tableName;
-  protected Set<HRegionInfo> unservedRegions;
+  private final Set<MetaRegion> metaRegions;
+  protected final byte [] tableName;
+  protected final Set<HRegionInfo> unservedRegions = new HashSet<HRegionInfo>();
   protected HMaster master;
-  protected final int numRetries;
-  protected final Sleeper sleeper;
-  
-  protected TableOperation(final HMaster master, final byte [] tableName) 
+
+  protected TableOperation(final HMaster master, final byte [] tableName)
   throws IOException {
-    this.sleeper = master.sleeper;
-    this.numRetries = master.numRetries;
-    
     this.master = master;
-    
     if (!this.master.isMasterRunning()) {
       throw new MasterNotRunningException();
     }
-
     this.tableName = tableName;
-    this.unservedRegions = new HashSet<HRegionInfo>();
 
     // We can not access any meta region if they have not already been
     // assigned and scanned.
-
     if (master.regionManager.metaScannerThread.waitForMetaRegionsOrClose()) {
       // We're shutting down. Forget it.
       throw new MasterNotRunningException(); 
     }
-
     this.metaRegions = master.regionManager.getMetaRegionsForTable(tableName);
   }
 
@@ -147,7 +130,7 @@
       postProcessMeta(m, server);
       unservedRegions.clear();
 
-      return true;
+      return Boolean.TRUE;
     }
   }
 
@@ -178,4 +161,4 @@
 
   protected abstract void postProcessMeta(MetaRegion m,
     HRegionInterface server) throws IOException;
-}
+}
\ No newline at end of file

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=662146&r1=662145&r2=662146&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 Sat May 31 21:42:11 2008
@@ -1009,12 +1009,17 @@
            this.memcacheSize.set(0);
         }
       }
-    } catch (IOException e) {
+    } catch (Throwable t) {
       // An exception here means that the snapshot was not persisted.
       // The hlog needs to be replayed so its content is restored to memcache.
       // Currently, only a server restart will do this.
+      // We used to only catch IOEs but its possible that we'd get other
+      // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
+      // all and sundry.
       this.log.abortCacheFlush();
-      throw new DroppedSnapshotException(e.getMessage());
+      DroppedSnapshotException dse = new DroppedSnapshotException();
+      dse.initCause(t);
+      throw dse;
     }
 
     // If we get to here, the HStores have been written. If we get an

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java?rev=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java Sat May 31 21:42:11 2008
@@ -74,7 +74,9 @@
   private void openReaders(final byte [] firstRow) throws IOException {
     if (this.readers != null) {
       for (int i = 0; i < this.readers.length; i++) {
-        this.readers[i].close();
+        if (this.readers[i] != null) {
+          this.readers[i].close();
+        }
       }
     }
     // Open our own copies of the Readers here inside in the scanner.
@@ -247,7 +249,7 @@
    * @param firstRow seek to this row
    * @return true if this is the first row or if the row was not found
    */
-  boolean findFirstRow(int i, final byte [] firstRow) throws IOException {
+  private boolean findFirstRow(int i, final byte [] firstRow) throws IOException {
     ImmutableBytesWritable ibw = new ImmutableBytesWritable();
     HStoreKey firstKey
       = (HStoreKey)readers[i].getClosest(new HStoreKey(firstRow), ibw);
@@ -276,7 +278,7 @@
    * @param i which reader to fetch next value from
    * @return true if there is more data available
    */
-  boolean getNext(int i) throws IOException {
+  private boolean getNext(int i) throws IOException {
     boolean result = false;
     ImmutableBytesWritable ibw = new ImmutableBytesWritable();
     long now = System.currentTimeMillis();
@@ -302,7 +304,7 @@
   }
   
   /** Close down the indicated reader. */
-  void closeSubScanner(int i) {
+  private void closeSubScanner(int i) {
     try {
       if(readers[i] != null) {
         try {
@@ -355,4 +357,4 @@
       this.lock.writeLock().unlock();
     }
   }
-}
\ No newline at end of file
+}

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=662146&r1=662145&r2=662146&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 Sat May 31 21:42:11 2008
@@ -22,28 +22,36 @@
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 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;
 import org.apache.hadoop.fs.Path;
 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.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;
 
 /**
- * Contains utility methods for manipulating HBase meta tables
+ * 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.).
  */
 public class MetaUtils {
   private static final Log LOG = LogFactory.getLog(MetaUtils.class);
@@ -209,7 +217,6 @@
     }
     
     // Open root region so we can scan it
-
     if (this.rootRegion == null) {
       openRootRegion();
     }
@@ -261,7 +268,7 @@
     HRegion metaRegion = openMetaRegion(metaRegionInfo);
     scanMetaRegion(metaRegion, listener);
   }
-  
+
   /**
    * Scan the passed in metaregion <code>m</code> invoking the passed
    * <code>listener</code> per row found.
@@ -269,8 +276,7 @@
    * @param listener
    * @throws IOException
    */
-  public void scanMetaRegion(final HRegion m,
-      final ScannerListener listener)
+  public void scanMetaRegion(final HRegion m, final ScannerListener listener)
   throws IOException {
     InternalScanner metaScanner = m.getScanner(HConstants.COL_REGIONINFO_ARRAY,
       HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null);
@@ -295,13 +301,13 @@
       metaScanner.close();
     }
   }
-  
+
   private void openRootRegion() throws IOException {
     this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO,
         this.rootdir, this.log, this.conf);
     this.rootRegion.compactStores();
   }
-  
+
   private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException {
     HRegion meta =
       HRegion.openHRegion(metaInfo, this.rootdir, this.log, this.conf);
@@ -339,4 +345,99 @@
     b.delete(HConstants.COL_STARTCODE);
     t.commit(b);
   }
+  
+  /**
+   * @param tableName
+   * @param hcd Add this column to <code>tableName</code>
+   * @throws IOException 
+   */
+  public void addColumn(final byte [] tableName,
+      final HColumnDescriptor hcd)
+  throws IOException {
+    List<HRegionInfo> metas = getMETARowsInROOT();
+    for (HRegionInfo hri: metas) {
+      final HRegion m = getMetaRegion(hri);
+      scanMetaRegion(m, new ScannerListener() {
+        private boolean inTable = false;
+        
+        @SuppressWarnings("synthetic-access")
+        public boolean processRow(HRegionInfo info) throws IOException {
+          if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
+            this.inTable = true;
+            info.getTableDesc().addFamily(hcd);
+            updateMETARegionInfo(m, info);
+            return false;
+          }
+          // If we got here and we have not yet encountered the table yet,
+          // inTable will be false.  Otherwise, we've passed out the table.
+          // Stop the scanner.
+          return this.inTable;
+        }});
+    }
+  }
+  
+  /**
+   * @param tableName
+   * @param columnFamily Name of column name to remove.
+   * @throws IOException
+   */
+  public void deleteColumn(final byte [] tableName,
+      final byte [] columnFamily) throws IOException {
+    List<HRegionInfo> metas = getMETARowsInROOT();
+    for (HRegionInfo hri: metas) {
+      final HRegion m = getMetaRegion(hri);
+      scanMetaRegion(m, new ScannerListener() {
+        private boolean inTable = false;
+        
+        @SuppressWarnings("synthetic-access")
+        public boolean processRow(HRegionInfo info) throws IOException {
+          if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
+            this.inTable = true;
+            info.getTableDesc().removeFamily(columnFamily);
+            updateMETARegionInfo(m, info);
+            return false;
+          }
+          // If we got here and we have not yet encountered the table yet,
+          // inTable will be false.  Otherwise, we've passed out the table.
+          // Stop the scanner.
+          return this.inTable;
+        }});
+    }
+  }
+  
+  private void updateMETARegionInfo(HRegion r, final HRegionInfo hri) 
+  throws IOException {
+    BatchUpdate b = new BatchUpdate(hri.getRegionName());
+    b.put(HConstants.COL_REGIONINFO, Writables.getBytes(hri));
+    r.batchUpdate(b);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Updated " + Bytes.toString(HConstants.COL_REGIONINFO) +
+        " column in row " + hri.getRegionName() + " in " +
+        Bytes.toString(r.getRegionName()));
+    }
+  }
+
+  /**
+   * @return List of <code>.META.<code> {@link HRegionInfo} found in the
+   * <code>-ROOT-</code> table.
+   * @throws IOException
+   * @see #getMetaRegion(HRegionInfo)
+   */
+  public List<HRegionInfo> getMETARowsInROOT() throws IOException {
+    if (!initialized) {
+      throw new IllegalStateException("Must call initialize method first.");
+    }
+    final List<HRegionInfo> result = new ArrayList<HRegionInfo>();
+    scanRootRegion(new ScannerListener() {
+      @SuppressWarnings("unused")
+      public boolean processRow(HRegionInfo info) throws IOException {
+        if (Bytes.equals(info.getTableDesc().getName(),
+            HConstants.META_TABLE_NAME)) {
+          result.add(info);
+          return false;
+        }
+        return true;
+      }});
+    return result;
+  }
 }
\ No newline at end of file

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=662146&r1=662145&r2=662146&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml (original)
+++ hadoop/hbase/trunk/src/webapps/master/WEB-INF/web.xml Sat May 31 21:42:11 2008
@@ -10,6 +10,11 @@
 
 
     <servlet>
+        <servlet-name>org.apache.hadoop.hbase.generated.master.loader_jsp</servlet-name>
+        <servlet-class>org.apache.hadoop.hbase.generated.master.loader_jsp</servlet-class>
+    </servlet>
+
+    <servlet>
         <servlet-name>org.apache.hadoop.hbase.generated.master.master_jsp</servlet-name>
         <servlet-class>org.apache.hadoop.hbase.generated.master.master_jsp</servlet-class>
     </servlet>
@@ -20,6 +25,11 @@
     </servlet>
 
     <servlet-mapping>
+        <servlet-name>org.apache.hadoop.hbase.generated.master.loader_jsp</servlet-name>
+        <url-pattern>/loader.jsp</url-pattern>
+    </servlet-mapping>
+
+    <servlet-mapping>
         <servlet-name>org.apache.hadoop.hbase.generated.master.master_jsp</servlet-name>
         <url-pattern>/master.jsp</url-pattern>
     </servlet-mapping>