You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2013/07/28 08:04:03 UTC

svn commit: r1507768 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/rest/ main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/hadoop/hbase/client/ test/java/org/apache/hadoop/hbase/ma...

Author: larsh
Date: Sun Jul 28 06:04:02 2013
New Revision: 1507768

URL: http://svn.apache.org/r1507768
Log:
HBASE-8698 potential thread creation in MetaScanner.metaScan

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HTable.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Sun Jul 28 06:04:02 2013
@@ -103,6 +103,7 @@ public class HBaseAdmin implements Abort
   private boolean aborted;
 
   private static volatile boolean synchronousBalanceSwitchSupported = true;
+  private final boolean cleanupConnectionOnClose; // close the connection in close()
 
   /**
    * Constructor
@@ -119,6 +120,7 @@ public class HBaseAdmin implements Abort
     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
     this.retryLongerMultiplier = this.conf.getInt(
         "hbase.client.retries.longer.multiplier", 10);
+    this.cleanupConnectionOnClose = true;
 
     int tries = 0;
     while ( true ){
@@ -165,6 +167,7 @@ public class HBaseAdmin implements Abort
       throws MasterNotRunningException, ZooKeeperConnectionException {
     this.conf = connection.getConfiguration();
     this.connection = connection;
+    this.cleanupConnectionOnClose = false;
 
     this.pause = this.conf.getLong("hbase.client.pause", 1000);
     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
@@ -454,7 +457,7 @@ public class HBaseAdmin implements Abort
             return true;
           }
         };
-        MetaScanner.metaScan(conf, visitor, desc.getName());
+        MetaScanner.metaScan(conf, connection, visitor, desc.getName());
         if (actualRegCount.get() != numRegs) {
           if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
             throw new RegionOfflineException("Only " + actualRegCount.get() +
@@ -1675,7 +1678,7 @@ public class HBaseAdmin implements Abort
         }
       };
 
-      MetaScanner.metaScan(conf, visitor);
+      MetaScanner.metaScan(conf, connection, visitor, null);
       pair = result.get();
     }
     return pair;
@@ -1802,7 +1805,7 @@ public class HBaseAdmin implements Abort
   }
 
   public void close() throws IOException {
-    if (this.connection != null) {
+    if (cleanupConnectionOnClose && this.connection != null) {
       this.connection.close();
     }
   }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Sun Jul 28 06:04:02 2013
@@ -802,7 +802,7 @@ public class HConnectionManager {
           return true;
         }
       };
-      MetaScanner.metaScan(conf, visitor);
+      MetaScanner.metaScan(conf, this, visitor, null);
       return available.get() && (regionCount.get() > 0);
     }
 
@@ -842,8 +842,8 @@ public class HConnectionManager {
     @Override
     public List<HRegionLocation> locateRegions(final byte[] tableName, final boolean useCache,
         final boolean offlined) throws IOException {
-      NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, tableName,
-        offlined);
+      NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, this,
+          tableName, offlined);
       final List<HRegionLocation> locations = new ArrayList<HRegionLocation>();
       for (HRegionInfo regionInfo : regions.keySet()) {
         locations.add(locateRegion(tableName, regionInfo.getStartKey(), useCache, true));
@@ -955,8 +955,8 @@ public class HConnectionManager {
       };
       try {
         // pre-fetch certain number of regions info at region cache.
-        MetaScanner.metaScan(conf, visitor, tableName, row,
-            this.prefetchRegionLimit);
+        MetaScanner.metaScan(conf, this, visitor, tableName, row,
+            this.prefetchRegionLimit, HConstants.META_TABLE_NAME);
       } catch (IOException e) {
         LOG.warn("Encountered problems when prefetch META table: ", e);
       }

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HTable.java Sun Jul 28 06:04:02 2013
@@ -25,7 +25,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.lang.reflect.Proxy;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Collections;
@@ -33,10 +32,8 @@ import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -210,9 +207,6 @@ public class HTable implements HTableInt
    */
   public HTable(final byte[] tableName, final HConnection connection, 
       final ExecutorService pool) throws IOException {
-    if (pool == null || pool.isShutdown()) {
-      throw new IllegalArgumentException("Pool is null or shut down.");
-    }
     if (connection == null || connection.isClosed()) {
       throw new IllegalArgumentException("Connection is null or closed.");
     }
@@ -481,7 +475,7 @@ public class HTable implements HTableInt
    * @throws IOException if a remote or network exception occurs
    */
   public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
-    return MetaScanner.allTableRegions(getConfiguration(), getTableName(), false);
+    return MetaScanner.allTableRegions(getConfiguration(), this.connection, getTableName(), false);
   }
 
   /**

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Sun Jul 28 06:04:02 2013
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
 import org.apache.hadoop.hbase.errorhandling.TimeoutException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
@@ -65,23 +64,23 @@ public class MetaScanner {
   public static void metaScan(Configuration configuration,
       MetaScannerVisitor visitor)
   throws IOException {
-    metaScan(configuration, visitor, null);
+    metaScan(configuration, null, visitor, null);
   }
 
   /**
-   * Scans the meta table and calls a visitor on each RowResult. Uses a table
-   * name to locate meta regions.
+   * Scans the meta table and calls a visitor on each RowResult and uses a empty
+   * start row value as table name.
    *
-   * @param configuration config
-   * @param visitor visitor object
-   * @param userTableName User table name in meta table to start scan at.  Pass
-   * null if not interested in a particular table.
+   * @param configuration conf
+   * @param connection connection to be used internally (null not allowed)
+   * @param visitor A custom visitor
    * @throws IOException e
    */
-  public static void metaScan(Configuration configuration,
+  public static void metaScan(Configuration configuration, HConnection connection,
       MetaScannerVisitor visitor, byte [] userTableName)
   throws IOException {
-    metaScan(configuration, visitor, userTableName, null, Integer.MAX_VALUE);
+    metaScan(configuration, connection, visitor, userTableName, null, Integer.MAX_VALUE,
+        HConstants.META_TABLE_NAME);
   }
 
   /**
@@ -103,7 +102,7 @@ public class MetaScanner {
       MetaScannerVisitor visitor, byte [] userTableName, byte[] row,
       int rowLimit)
   throws IOException {
-    metaScan(configuration, visitor, userTableName, row, rowLimit,
+    metaScan(configuration, null, visitor, userTableName, row, rowLimit,
       HConstants.META_TABLE_NAME);
   }
 
@@ -113,6 +112,7 @@ public class MetaScanner {
    * <code>rowLimit</code> of rows.
    *
    * @param configuration HBase configuration.
+   * @param connection connection to be used internally (null not allowed)
    * @param visitor Visitor object. Closes the visitor before returning.
    * @param tableName User table name in meta table to start scan at.  Pass
    * null if not interested in a particular table.
@@ -123,42 +123,28 @@ public class MetaScanner {
    * @param metaTableName Meta table to scan, root or meta.
    * @throws IOException e
    */
-  public static void metaScan(Configuration configuration,
-      final MetaScannerVisitor visitor, final byte[] tableName,
-      final byte[] row, final int rowLimit, final byte[] metaTableName)
-      throws IOException {
-    try {
-      HConnectionManager.execute(new HConnectable<Void>(configuration) {
-        @Override
-        public Void connect(HConnection connection) throws IOException {
-          metaScan(conf, connection, visitor, tableName, row, rowLimit,
-              metaTableName);
-          return null;
-        }
-      });
-    } finally {
-      visitor.close();
-    }
-  }
-
-  private static void metaScan(Configuration configuration, HConnection connection,
+  public static void metaScan(Configuration configuration, HConnection connection,
       MetaScannerVisitor visitor, byte [] tableName, byte[] row,
       int rowLimit, final byte [] metaTableName)
   throws IOException {
-    int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
-
-    // if row is not null, we want to use the startKey of the row's region as
-    // the startRow for the meta scan.
-    byte[] startRow;
-    if (row != null) {
-      // Scan starting at a particular row in a particular table
-      assert tableName != null;
-      byte[] searchRow =
-        HRegionInfo.createRegionName(tableName, row, HConstants.NINES,
-          false);
-      HTable metaTable = null;
-      try {
-        metaTable = new HTable(configuration, HConstants.META_TABLE_NAME);
+    HTable metaTable = null;
+    try {
+      if (connection == null) {
+        metaTable = new HTable(configuration, HConstants.META_TABLE_NAME, null);
+      } else {
+        metaTable = new HTable(HConstants.META_TABLE_NAME, connection, null);
+      }
+      int rowUpperLimit = rowLimit > 0 ? rowLimit: Integer.MAX_VALUE;
+
+      // if row is not null, we want to use the startKey of the row's region as
+      // the startRow for the meta scan.
+      byte[] startRow;
+      if (row != null) {
+        // Scan starting at a particular row in a particular table
+        assert tableName != null;
+        byte[] searchRow =
+          HRegionInfo.createRegionName(tableName, row, HConstants.NINES,
+            false);
         Result startRowResult = metaTable.getRowOrBefore(searchRow,
             HConstants.CATALOG_FAMILY);
         if (startRowResult == null) {
@@ -176,80 +162,72 @@ public class MetaScanner {
         byte[] rowBefore = regionInfo.getStartKey();
         startRow = HRegionInfo.createRegionName(tableName, rowBefore,
             HConstants.ZEROES, false);
-      } finally {
-        if (metaTable != null) {
-          metaTable.close();
+      } else if (tableName == null || tableName.length == 0) {
+        // Full META scan
+        startRow = HConstants.EMPTY_START_ROW;
+      } else {
+        // Scan META for an entire table
+        startRow = HRegionInfo.createRegionName(
+            tableName, HConstants.EMPTY_START_ROW, HConstants.ZEROES, false);
+      }
+
+      // Scan over each meta region
+      ScannerCallable callable;
+      int rows = Math.min(rowLimit, configuration.getInt(
+          HConstants.HBASE_META_SCANNER_CACHING,
+          HConstants.DEFAULT_HBASE_META_SCANNER_CACHING));
+      do {
+        final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Scanning " + Bytes.toString(metaTableName) +
+            " starting at row=" + Bytes.toStringBinary(startRow) + " for max=" +
+            rowUpperLimit + " rows using " + metaTable.getConnection().toString());
         }
-      }
-    } else if (tableName == null || tableName.length == 0) {
-      // Full META scan
-      startRow = HConstants.EMPTY_START_ROW;
-    } else {
-      // Scan META for an entire table
-      startRow = HRegionInfo.createRegionName(
-          tableName, HConstants.EMPTY_START_ROW, HConstants.ZEROES, false);
-    }
+        callable = new ScannerCallable(metaTable.getConnection(), metaTableName, scan, null);
+        // Open scanner
+        callable.withRetries();
 
-    // Scan over each meta region
-    ScannerCallable callable;
-    int rows = Math.min(rowLimit, configuration.getInt(
-        HConstants.HBASE_META_SCANNER_CACHING,
-        HConstants.DEFAULT_HBASE_META_SCANNER_CACHING));
-    do {
-      final Scan scan = new Scan(startRow).addFamily(HConstants.CATALOG_FAMILY);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Scanning " + Bytes.toString(metaTableName) +
-          " starting at row=" + Bytes.toStringBinary(startRow) + " for max=" +
-          rowUpperLimit + " rows using " + connection.toString());
-      }
-      callable = new ScannerCallable(connection, metaTableName, scan, null);
-      // Open scanner
-      callable.withRetries();
-
-      int processedRows = 0;
-      try {
-        callable.setCaching(rows);
-        done: do {
-          if (processedRows >= rowUpperLimit) {
-            break;
-          }
-          //we have all the rows here
-          Result [] rrs = callable.withRetries();
-          if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) {
-            break; //exit completely
-          }
-          for (Result rr : rrs) {
+        int processedRows = 0;
+        try {
+          callable.setCaching(rows);
+          done: do {
             if (processedRows >= rowUpperLimit) {
-              break done;
+              break;
             }
-            if (!visitor.processRow(rr))
-              break done; //exit completely
-            processedRows++;
-          }
-          //here, we didn't break anywhere. Check if we have more rows
-        } while(true);
-        // Advance the startRow to the end key of the current region
-        startRow = callable.getHRegionInfo().getEndKey();
-      } finally {
-        // Close scanner
-        callable.setClose();
-        callable.withRetries();
+            //we have all the rows here
+            Result [] rrs = callable.withRetries();
+            if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) {
+              break; //exit completely
+            }
+            for (Result rr : rrs) {
+              if (processedRows >= rowUpperLimit) {
+                break done;
+              }
+              if (!visitor.processRow(rr))
+                break done; //exit completely
+              processedRows++;
+            }
+            //here, we didn't break anywhere. Check if we have more rows
+          } while(true);
+          // Advance the startRow to the end key of the current region
+          startRow = callable.getHRegionInfo().getEndKey();
+        } finally {
+          // Close scanner
+          callable.setClose();
+          callable.withRetries();
+        }
+      } while (Bytes.compareTo(startRow, HConstants.LAST_ROW) != 0);
+    } finally {
+      visitor.close();
+      if (metaTable != null) {
+        metaTable.close();
       }
-    } while (Bytes.compareTo(startRow, HConstants.LAST_ROW) != 0);
-  }
-
-  /**
-   * Lists all of the regions currently in META.
-   * @param conf
-   * @return List of all user-space regions.
-   * @throws IOException
-   */
-  public static List<HRegionInfo> listAllRegions(Configuration conf)
-  throws IOException {
-    return listAllRegions(conf, true);
+    }
   }
 
   /**
+   * Used in tests.
+   *
    * Lists all of the regions currently in META.
    * @param conf
    * @param offlined True if we are to include offlined regions, false and we'll
@@ -286,13 +264,14 @@ public class MetaScanner {
   /**
    * Lists all of the table regions currently in META.
    * @param conf
+   * @param connection connection to be used internally (null to create a new connection)
    * @param offlined True if we are to include offlined regions, false and we'll
    * leave out offlined regions from returned list.
    * @return Map of all user-space regions to servers
    * @throws IOException
    */
   public static NavigableMap<HRegionInfo, ServerName> allTableRegions(Configuration conf,
-      final byte [] tablename, final boolean offlined) throws IOException {
+      HConnection connection, final byte[] tablename, final boolean offlined) throws IOException {
     final NavigableMap<HRegionInfo, ServerName> regions =
       new TreeMap<HRegionInfo, ServerName>();
     MetaScannerVisitor visitor = new TableMetaScannerVisitor(conf, tablename) {
@@ -321,7 +300,7 @@ public class MetaScanner {
         return true;
       }
     };
-    metaScan(conf, visitor, tablename);
+    metaScan(conf, connection, visitor, tablename);
     return regions;
   }
 

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java Sun Jul 28 06:04:02 2013
@@ -76,7 +76,7 @@ public class RegionsResource extends Res
       String tableName = tableResource.getName();
       TableInfoModel model = new TableInfoModel(tableName);
       Map<HRegionInfo,ServerName> regions = MetaScanner.allTableRegions(
-        servlet.getConfiguration(), Bytes.toBytes(tableName), false);
+        servlet.getConfiguration(), null, Bytes.toBytes(tableName), false);
       for (Map.Entry<HRegionInfo,ServerName> e: regions.entrySet()) {
         HRegionInfo hri = e.getKey();
         ServerName addr = e.getValue();

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java Sun Jul 28 06:04:02 2013
@@ -2593,7 +2593,7 @@ public class HBaseFsck extends Configure
     };
 
     // Scan -ROOT- to pick up META regions
-    MetaScanner.metaScan(getConf(), visitor, null, null,
+    MetaScanner.metaScan(getConf(), null, visitor, null, null,
       Integer.MAX_VALUE, HConstants.ROOT_TABLE_NAME);
 
     if (!checkMetaOnly) {

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java Sun Jul 28 06:04:02 2013
@@ -86,7 +86,7 @@ public class TestMetaScanner {
     doReturn(true).when(visitor).processRow((Result)anyObject());
 
     // Scanning the entire table should give us three rows
-    MetaScanner.metaScan(conf, visitor, TABLENAME);
+    MetaScanner.metaScan(conf, null, visitor, TABLENAME);
     verify(visitor, times(3)).processRow((Result)anyObject());
 
     // Scanning the table with a specified empty start row should also
@@ -192,7 +192,7 @@ public class TestMetaScanner {
          while(!isStopped()) {
            try {
             NavigableMap<HRegionInfo, ServerName> regions =
-                MetaScanner.allTableRegions(TEST_UTIL.getConfiguration(), TABLENAME, false);
+                MetaScanner.allTableRegions(TEST_UTIL.getConfiguration(), null, TABLENAME, false);
 
             LOG.info("-------");
             byte[] lastEndKey = HConstants.EMPTY_START_ROW;

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java Sun Jul 28 06:04:02 2013
@@ -102,7 +102,7 @@ public class TestRestartCluster {
     }
 
     List<HRegionInfo> allRegions =
-      MetaScanner.listAllRegions(UTIL.getConfiguration());
+      MetaScanner.listAllRegions(UTIL.getConfiguration(), true);
     assertEquals(3, allRegions.size());
 
     LOG.info("\n\nShutting down cluster");
@@ -118,7 +118,7 @@ public class TestRestartCluster {
     // Otherwise we're reusing an HConnection that has gone stale because
     // the shutdown of the cluster also called shut of the connection.
     allRegions = MetaScanner.
-      listAllRegions(new Configuration(UTIL.getConfiguration()));
+      listAllRegions(new Configuration(UTIL.getConfiguration()), true);
     assertEquals(3, allRegions.size());
 
     LOG.info("\n\nWaiting for tables to be available");

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java?rev=1507768&r1=1507767&r2=1507768&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java Sun Jul 28 06:04:02 2013
@@ -214,7 +214,8 @@ public class TestEndToEndSplitTransactio
       try {
         Random random = new Random();
         for (int i=0; i< 5; i++) {
-          NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, tableName, false);
+          NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, null,
+              tableName, false);
           if (regions.size() == 0) {
             continue;
           }
@@ -286,8 +287,8 @@ public class TestEndToEndSplitTransactio
     void verifyRegionsUsingMetaScanner() throws Exception {
 
       //MetaScanner.allTableRegions()
-      NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, tableName,
-          false);
+      NavigableMap<HRegionInfo, ServerName> regions = MetaScanner.allTableRegions(conf, null,
+          tableName, false);
       verifyTableRegions(regions.keySet());
 
       //MetaScanner.listAllRegions()