You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ji...@apache.org on 2008/06/28 01:13:26 UTC

svn commit: r672423 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/master/

Author: jimk
Date: Fri Jun 27 16:13:26 2008
New Revision: 672423

URL: http://svn.apache.org/viewvc?rev=672423&view=rev
Log:
HBASE-627   Disable table doesn't work reliably

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/TableOperation.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=672423&r1=672422&r2=672423&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Fri Jun 27 16:13:26 2008
@@ -71,6 +71,7 @@
    HBASE-710   If clocks are way off, then we can have daughter split come
                before rather than after its parent in .META.
    HBASE-714   Showing bytes in log when should be string (2)
+   HBASE-627   Disable table doesn't work reliably
    
    
   IMPROVEMENTS

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=672423&r1=672422&r2=672423&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Fri Jun 27 16:13:26 2008
@@ -21,7 +21,6 @@
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.NoSuchElementException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -314,78 +313,17 @@
       throw new MasterNotRunningException("master has been shut down");
     }
     HTableDescriptor.isLegalTableName(tableName);
-    HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
-    
     try {
       this.master.enableTable(tableName);
-      
     } catch (RemoteException e) {
       throw RemoteExceptionHandler.decodeRemoteException(e);
     }
 
-    // Wait until first region is enabled
+    // Wait until all regions are enabled
     
-    HRegionInterface server =
-      connection.getHRegionConnection(firstMetaServer.getServerAddress());
-
-    HRegionInfo info = new HRegionInfo();
-    for (int tries = 0; tries < numRetries; tries++) {
-      int valuesfound = 0;
-      long scannerId = -1L;
-      try {
-        scannerId =
-          server.openScanner(firstMetaServer.getRegionInfo().getRegionName(),
-            HConstants.COL_REGIONINFO_ARRAY, tableName,
-            HConstants.LATEST_TIMESTAMP, null);
-        boolean isenabled = false;
-        
-        while (true) {
-          RowResult values = server.next(scannerId);
-          if (values == null || values.size() == 0) {
-            if (valuesfound == 0) {
-              throw new NoSuchElementException(
-                  "table " + Bytes.toString(tableName) + " not found");
-            }
-            break;
-          }
-          valuesfound += 1;
-          for (Map.Entry<byte [], Cell> e: values.entrySet()) {
-            if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) {
-              info = (HRegionInfo) Writables.getWritable(
-                e.getValue().getValue(), info);
-            
-              isenabled = !info.isOffline();
-              break;
-            }
-          }
-          if (isenabled) {
-            break;
-          }
-        }
-        if (isenabled) {
-          break;
-        }
-        
-      } catch (IOException e) {
-        if (tries == numRetries - 1) {                  // no more retries
-          if (e instanceof RemoteException) {
-            e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-          }
-          throw e;
-        }
-        
-      } finally {
-        if (scannerId != -1L) {
-          try {
-            server.close(scannerId);
-            
-          } catch (Exception e) {
-            LOG.warn(e);
-          }
-        }
-      }
+    while (!isTableEnabled(tableName)) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Sleep. Waiting for first region to be enabled from " +
+        LOG.debug("Sleep. Waiting for all regions to be enabled from " +
           Bytes.toString(tableName));
       }
       try {
@@ -395,7 +333,7 @@
         // continue
       }
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Wake. Waiting for first region to be enabled from " +
+        LOG.debug("Wake. Waiting for all regions to be enabled from " +
           Bytes.toString(tableName));
       }
     }
@@ -436,91 +374,59 @@
       throw new MasterNotRunningException("master has been shut down");
     }
     HTableDescriptor.isLegalTableName(tableName);
-    HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
     try {
       this.master.disableTable(tableName);
     } catch (RemoteException e) {
       throw RemoteExceptionHandler.decodeRemoteException(e);
     }
 
-    // Wait until first region is disabled
+    // Wait until all regions are disabled
     
-    HRegionInterface server =
-      connection.getHRegionConnection(firstMetaServer.getServerAddress());
-
-    HRegionInfo info = new HRegionInfo();
-    for(int tries = 0; tries < numRetries; tries++) {
-      int valuesfound = 0;
-      long scannerId = -1L;
-      try {
-        scannerId =
-          server.openScanner(firstMetaServer.getRegionInfo().getRegionName(),
-              HConstants.COL_REGIONINFO_ARRAY, tableName,
-              HConstants.LATEST_TIMESTAMP, null);
-        boolean disabled = false;
-        while (true) {
-          RowResult values = server.next(scannerId);
-          if (values == null || values.size() == 0) {
-            if (valuesfound == 0) {
-              throw new NoSuchElementException("table " +
-                Bytes.toString(tableName) + " not found");
-            }
-            break;
-          }
-          valuesfound += 1;
-          for (Map.Entry<byte [], Cell> e: values.entrySet()) {
-            if (Bytes.equals(e.getKey(), HConstants.COL_REGIONINFO)) {
-              info = (HRegionInfo) Writables.getWritable(
-                e.getValue().getValue(), info);
-            
-              disabled = info.isOffline();
-              break;
-            }
-          }
-          if (disabled) {
-            break;
-          }
-        }
-        if (disabled) {
-          break;
-        }
-        
-      } catch (IOException e) {
-        if (tries == numRetries - 1) {                  // no more retries
-          if (e instanceof RemoteException) {
-            e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
-          }
-          throw e;
-        }
-        
-      } finally {
-        if (scannerId != -1L) {
-          try {
-            server.close(scannerId);
-            
-          } catch (Exception e) {
-            LOG.warn(e);
-          }
-        }
-      }
+    while (isTableEnabled(tableName)) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Sleep. Waiting for first region to be disabled from " +
+        LOG.debug("Sleep. Waiting for all regions to be disabled from " +
           Bytes.toString(tableName));
       }
       try {
         Thread.sleep(pause);
+        
       } catch (InterruptedException e) {
         // continue
       }
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Wake. Waiting for first region to be disabled from " +
-            tableName);
+        LOG.debug("Wake. Waiting for all regions to be disabled from " +
+          Bytes.toString(tableName));
       }
     }
     LOG.info("Disabled " + Bytes.toString(tableName));
   }
   
   /**
+   * @param tableName name of table to check
+   * @return true if table is on-line
+   * @throws IOException
+   */
+  public boolean isTableEnabled(Text tableName) throws IOException {
+    return isTableEnabled(tableName.getBytes());
+  }
+  /**
+   * @param tableName name of table to check
+   * @return true if table is on-line
+   * @throws IOException
+   */
+  public boolean isTableEnabled(String tableName) throws IOException {
+    return isTableEnabled(Bytes.toBytes(tableName));
+  }
+  /**
+   * @param tableName name of table to check
+   * @return true if table is on-line
+   * @throws IOException
+   */
+  public boolean isTableEnabled(byte[] tableName) throws IOException {
+    return connection.isTableEnabled(tableName);
+  }
+  
+  /**
    * Add a column to an existing table
    * 
    * @param tableName name of the table to add column to

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java?rev=672423&r1=672422&r2=672423&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnection.java Fri Jun 27 16:13:26 2008
@@ -46,8 +46,17 @@
    * Checks if <code>tableName</code> exists.
    * @param tableName Table to check.
    * @return True if table exists already.
+   * @throws MasterNotRunningException
    */
-  public boolean tableExists(final byte [] tableName);
+  public boolean tableExists(final byte [] tableName)
+  throws MasterNotRunningException;
+
+  /**
+   * @param tableName
+   * @return true if the table is enabled, false otherwise
+   * @throws IOException
+   */
+  public boolean isTableEnabled(byte[] tableName) throws IOException;
   
   /**
    * List all the userspace tables.  In other words, scan the META table.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=672423&r1=672422&r2=672423&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HConnectionManager.java Fri Jun 27 16:13:26 2008
@@ -234,7 +234,9 @@
     }
 
     /** {@inheritDoc} */
-    public boolean tableExists(final byte [] tableName) {
+    public boolean tableExists(final byte [] tableName)
+    throws MasterNotRunningException {
+      getMaster();
       if (tableName == null) {
         throw new IllegalArgumentException("Table name cannot be null");
       }
@@ -269,12 +271,15 @@
     public HRegionLocation getRegionLocation(final byte [] name,
         final byte [] row, boolean reload)
     throws IOException {
+      getMaster();
       return reload? relocateRegion(name, row): locateRegion(name, row);
     }
 
     /** {@inheritDoc} */
     public HTableDescriptor[] listTables() throws IOException {
-      final HashSet<HTableDescriptor> uniqueTables = new HashSet<HTableDescriptor>();
+      getMaster();
+      final HashSet<HTableDescriptor> uniqueTables =
+        new HashSet<HTableDescriptor>();
 
       MetaScannerVisitor visitor = new MetaScannerVisitor() {
 
@@ -297,44 +302,132 @@
       return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
     }
 
-    /**
-     * @param tableName
-     * @return table metadata 
-     * @throws IOException
-     */
+    /** {@inheritDoc} */
+    public boolean isTableEnabled(byte[] tableName) throws IOException {
+      if (!tableExists(tableName)) {
+        throw new TableNotFoundException(Bytes.toString(tableName));
+      }
+      if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
+        // The root region is always enabled
+        return true;
+      }
+      
+      boolean result = true;
+      int rowsScanned = 0;
+      byte[] startKey =
+        HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES);
+      HRegionInfo currentRegion = null;
+      do {
+        if (currentRegion != null) {
+          byte[] endKey = currentRegion.getEndKey();
+          if (endKey == null ||
+              Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)) {
+            // We have reached the end of the table and we're done
+            break;
+          }
+        }
+        HRegionInfo oldRegion = currentRegion;
+        if (oldRegion != null) {
+          startKey = oldRegion.getEndKey();
+        }
+        ScannerCallable s = new ScannerCallable(this, 
+            (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
+                HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME),
+            HConstants.COL_REGIONINFO_ARRAY, startKey,
+            HConstants.LATEST_TIMESTAMP, null
+        );
+        // Open scanner
+        getRegionServerWithRetries(s);
+        currentRegion = s.getHRegionInfo();
+        try {
+          RowResult r = null;
+          while (result && (r = getRegionServerWithRetries(s)) != null) {
+            Cell c = r.get(HConstants.COL_REGIONINFO);
+            if (c != null) {
+              byte[] value = c.getValue();
+              if (value != null) {
+                HRegionInfo info = Writables.getHRegionInfoOrNull(value);
+                if (info != null) {
+                  if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
+                    rowsScanned += 1;
+                    result = !info.isOffline();
+                  }
+                }
+              }
+            }
+          }
+        } finally {
+          s.setClose();
+          getRegionServerWithRetries(s);
+        }
+      } while (result);
+      return rowsScanned > 0 && result;
+    }
+    
+    /** {@inheritDoc} */
     public HTableDescriptor getHTableDescriptor(byte[] tableName)
     throws IOException {
       if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
         return new UnmodifyableHTableDescriptor(HTableDescriptor.ROOT_TABLEDESC);
       }
-      HTable meta = new HTable(conf,
-          Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
-              HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME);
-      Scanner s = meta.getScanner(HConstants.COL_REGIONINFO_ARRAY,
-          HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES));
-      try {
-        RowResult r = null;
-        while ((r = s.next()) != null) {
-          Cell c = r.get(HConstants.COL_REGIONINFO);
-          if (c != null) {
-            HRegionInfo info = Writables.getHRegionInfoOrNull(c.getValue());
-            if (info != null) {
-              if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
-                return new UnmodifyableHTableDescriptor(info.getTableDesc());
+      if (!tableExists(tableName)) {
+        throw new TableNotFoundException(Bytes.toString(tableName));
+      }
+      byte[] startKey =
+        HRegionInfo.createRegionName(tableName, null, HConstants.ZEROES);
+      
+      HTableDescriptor result = null;
+      HRegionInfo currentRegion = null;
+      ScannerCallable s = null;
+      while (result == null) {
+        if (currentRegion != null) {
+          byte[] endKey = currentRegion.getEndKey();
+          if (endKey == null ||
+              Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY)) {
+            // We have reached the end of the table and we're done
+            break;
+          }
+        }
+        HRegionInfo oldRegion = currentRegion;
+        if (oldRegion != null) {
+          startKey = oldRegion.getEndKey();
+        }
+        s = new ScannerCallable(this, 
+            (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
+                HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME),
+            HConstants.COL_REGIONINFO_ARRAY, startKey,
+            HConstants.LATEST_TIMESTAMP, null
+        );
+        // Open scanner
+        getRegionServerWithRetries(s);
+        currentRegion = s.getHRegionInfo();
+        try {
+          RowResult r = null;
+          while ((r = getRegionServerWithRetries(s)) != null) {
+            Cell c = r.get(HConstants.COL_REGIONINFO);
+            if (c != null) {
+              HRegionInfo info = Writables.getHRegionInfoOrNull(c.getValue());
+              if (info != null) {
+                if (Bytes.equals(info.getTableDesc().getName(), tableName)) {
+                  result = new UnmodifyableHTableDescriptor(info.getTableDesc());
+                  break;
+                }
               }
             }
           }
+        } finally {
+          s.setClose();
+          getRegionServerWithRetries(s);
         }
-        return null;
-      } finally {
-        s.close();
       }
+      return result;
     }
 
     /** {@inheritDoc} */
     public HRegionLocation locateRegion(final byte [] tableName,
         final byte [] row)
     throws IOException{
+      getMaster();
       return locateRegion(tableName, row, true);
     }
 
@@ -342,6 +435,7 @@
     public HRegionLocation relocateRegion(final byte [] tableName,
         final byte [] row)
     throws IOException{
+      getMaster();
       return locateRegion(tableName, row, false);
     }
 
@@ -644,6 +738,7 @@
     /** {@inheritDoc} */
     public HRegionInterface getHRegionConnection(HServerAddress regionServer) 
     throws IOException {
+      getMaster();
       HRegionInterface server;
       synchronized (this.servers) {
         // See if we already have a connection
@@ -768,6 +863,7 @@
     /** {@inheritDoc} */
     public <T> T getRegionServerWithRetries(ServerCallable<T> callable) 
     throws IOException, RuntimeException {
+      getMaster();
       List<Throwable> exceptions = new ArrayList<Throwable>();
       for(int tries = 0; tries < numRetries; tries++) {
         try {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java?rev=672423&r1=672422&r2=672423&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java Fri Jun 27 16:13:26 2008
@@ -42,7 +42,6 @@
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -130,24 +129,24 @@
    * @return true if table is on-line
    * @throws IOException
    */
-  public static boolean isTableOnline(Text tableName) throws IOException {
-    return isTableOnline(tableName.getBytes());
+  public static boolean isTableEnabled(Text tableName) throws IOException {
+    return isTableEnabled(tableName.getBytes());
   }
   /**
    * @param tableName name of table to check
    * @return true if table is on-line
    * @throws IOException
    */
-  public static boolean isTableOnline(String tableName) throws IOException {
-    return isTableOnline(Bytes.toBytes(tableName));
+  public static boolean isTableEnabled(String tableName) throws IOException {
+    return isTableEnabled(Bytes.toBytes(tableName));
   }
   /**
    * @param tableName name of table to check
    * @return true if table is on-line
    * @throws IOException
    */
-  public static boolean isTableOnline(byte[] tableName) throws IOException {
-    return isTableOnline(new HBaseConfiguration(), tableName);
+  public static boolean isTableEnabled(byte[] tableName) throws IOException {
+    return isTableEnabled(new HBaseConfiguration(), tableName);
   }
   
   /**
@@ -156,9 +155,9 @@
    * @return true if table is on-line
    * @throws IOException
    */
-  public static boolean isTableOnline(HBaseConfiguration conf, Text tableName)
+  public static boolean isTableEnabled(HBaseConfiguration conf, Text tableName)
   throws IOException {
-    return isTableOnline(conf, tableName.getBytes());
+    return isTableEnabled(conf, tableName.getBytes());
   }
   
   /**
@@ -167,9 +166,9 @@
    * @return true if table is on-line
    * @throws IOException
    */
-  public static boolean isTableOnline(HBaseConfiguration conf, String tableName)
+  public static boolean isTableEnabled(HBaseConfiguration conf, String tableName)
   throws IOException {
-    return isTableOnline(conf, Bytes.toBytes(tableName));
+    return isTableEnabled(conf, Bytes.toBytes(tableName));
   }
 
   /**
@@ -178,121 +177,9 @@
    * @return true if table is on-line
    * @throws IOException
    */
-  public static boolean isTableOnline(HBaseConfiguration conf, byte[] tableName)
+  public static boolean isTableEnabled(HBaseConfiguration conf, byte[] tableName)
   throws IOException {
-    boolean online = true;
-    if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
-      // The root region is always on-line
-      return true;
-    }
-    HTable meta = new HTable(conf,
-        Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
-            HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME);
-    Scanner s = meta.getScanner(HConstants.COL_REGIONINFO_ARRAY,
-        HRegionInfo.createRegionName(tableName, null, HConstants.NINES));
-    try {
-      RowResult r = null;
-      while ((r = s.next()) != null) {
-        Cell c = r.get(HConstants.COL_REGIONINFO);
-        if (c != null) {
-          HRegionInfo info = Writables.getHRegionInfoOrNull(c.getValue());
-          if (info != null) {
-            if (info.isOffline()) {
-              online = false;
-              break;
-            }
-          }
-        }
-      }
-    } finally {
-      s.close();
-    }
-    return online;
-  }
-  
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
-   */
-  public static boolean isTableOffline(Text tableName) throws IOException {
-    return isTableOffline(tableName.getBytes());
-  }
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
-   */
-  public static boolean isTableOffline(String tableName) throws IOException {
-    return isTableOffline(Bytes.toBytes(tableName));
-  }
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
-   */
-  public static boolean isTableOffline(byte[] tableName) throws IOException {
-    return isTableOffline(new HBaseConfiguration(), tableName);
-  }
-  
-  /**
-   * @param conf HBaseConfiguration object
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
-   */
-  public static boolean isTableOffline(HBaseConfiguration conf, Text tableName)
-  throws IOException {
-    return isTableOffline(conf, tableName.getBytes());
-  }
-  
-  /**
-   * @param conf HBaseConfiguration object
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
-   */
-  public static boolean isTableOffline(HBaseConfiguration conf, String tableName)
-  throws IOException {
-    return isTableOffline(conf, Bytes.toBytes(tableName));
-  }
-
-  /**
-   * @param conf HBaseConfiguration object
-   * @param tableName name of table to check
-   * @return true if table is off-line
-   * @throws IOException
-   */
-  public static boolean isTableOffline(HBaseConfiguration conf, byte[] tableName)
-  throws IOException {
-    boolean offline = true;
-    if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
-      // The root region is always online
-      return false;
-    }
-    HTable meta = new HTable(conf,
-        Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
-            HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME);
-    Scanner s = meta.getScanner(HConstants.COL_REGIONINFO_ARRAY,
-        HRegionInfo.createRegionName(tableName, null, HConstants.NINES));
-    try {
-      RowResult r = null;
-      while ((r = s.next()) != null) {
-        Cell c = r.get(HConstants.COL_REGIONINFO);
-        if (c != null) {
-          HRegionInfo info = Writables.getHRegionInfoOrNull(c.getValue());
-          if (info != null) {
-            if (!info.isOffline()) {
-              offline = false;
-              break;
-            }
-          }
-        }
-      }
-    } finally {
-      s.close();
-    }
-    return offline;
+    return HConnectionManager.getConnection(conf).isTableEnabled(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=672423&r1=672422&r2=672423&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 Fri Jun 27 16:13:26 2008
@@ -97,7 +97,7 @@
           }
 
           tableExists = true;
-          if (!isBeingServed(serverName, startCode)) {
+          if (!isBeingServed(serverName, startCode) || !isEnabled(info)) {
             unservedRegions.add(info);
           }
           processScanItem(serverName, startCode, info);