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

svn commit: r1459130 - in /hbase/trunk: conf/ hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ hbase-server/src/main/java/org/a...

Author: sershe
Date: Thu Mar 21 02:04:07 2013
New Revision: 1459130

URL: http://svn.apache.org/r1459130
Log:
HBASE-7948 client doesn't need to refresh meta while the region is opening

Added:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionOpeningException.java
Modified:
    hbase/trunk/conf/log4j.properties
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
    hbase/trunk/hbase-server/src/test/resources/log4j.properties

Modified: hbase/trunk/conf/log4j.properties
URL: http://svn.apache.org/viewvc/hbase/trunk/conf/log4j.properties?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/conf/log4j.properties (original)
+++ hbase/trunk/conf/log4j.properties Thu Mar 21 02:04:07 2013
@@ -84,6 +84,9 @@ log4j.logger.org.apache.hadoop.hbase.zoo
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
 #log4j.logger.org.apache.hadoop.dfs=DEBUG
 # Set this class to log INFO only otherwise its OTT
+# Enable this to get detailed connection error/retry logging.
+# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+
 
 # Uncomment this line to enable tracing on _every_ RPC call (this can be a lot of output)
 #log4j.logger.org.apache.hadoop.ipc.HBaseServer.trace=DEBUG

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java Thu Mar 21 02:04:07 2013
@@ -40,6 +40,10 @@ public class HRegionLocation implements 
   // Cache of the hostname + port
   private String cachedHostnamePort;
 
+  public HRegionLocation(HRegionInfo regionInfo, ServerName serverName) {
+    this(regionInfo, serverName, HConstants.NO_SEQNUM);
+  }
+
   public HRegionLocation(HRegionInfo regionInfo, ServerName serverName, long seqNum) {
     this.regionInfo = regionInfo;
     this.serverName = serverName;

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Thu Mar 21 02:04:07 2013
@@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.client.co
 import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
 import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
+import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
 import org.apache.hadoop.hbase.exceptions.RegionServerStoppedException;
 import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
 import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.SoftValueSortedMap;
 import org.apache.hadoop.hbase.util.Triple;
@@ -1065,8 +1067,8 @@ public class HConnectionManager {
               return true; // don't cache it
             }
             // instantiate the location
-            HRegionLocation loc = new HRegionLocation(regionInfo, serverName,
-                HRegionInfo.getSeqNumDuringOpen(result));
+            long seqNum = HRegionInfo.getSeqNumDuringOpen(result);
+            HRegionLocation loc = new HRegionLocation(regionInfo, serverName, seqNum);
             // cache this meta entry
             cacheLocation(tableName, null, loc);
             return true;
@@ -1196,7 +1198,7 @@ public class HConnectionManager {
 
           // Instantiate the location
           location = new HRegionLocation(regionInfo, serverName,
-              HRegionInfo.getSeqNumDuringOpen(regionInfoRow));
+            HRegionInfo.getSeqNumDuringOpen(regionInfoRow));
           cacheLocation(tableName, null, location);
           return location;
         } catch (TableNotFoundException e) {
@@ -1292,8 +1294,7 @@ public class HConnectionManager {
     void forceDeleteCachedLocation(final byte [] tableName, final byte [] row) {
       HRegionLocation rl = null;
       synchronized (this.cachedRegionLocations) {
-        Map<byte[], HRegionLocation> tableLocations =
-            getTableLocations(tableName);
+        Map<byte[], HRegionLocation> tableLocations = getTableLocations(tableName);
         // start to examine the cache. we can only do cache actions
         // if there's something in the cache for this table.
         if (!tableLocations.isEmpty()) {
@@ -1853,7 +1854,7 @@ public class HConnectionManager {
     */
     void deleteCachedLocation(HRegionInfo hri, HRegionLocation source) {
       boolean isStaleDelete = false;
-      HRegionLocation oldLocation;
+      HRegionLocation oldLocation = null;
       synchronized (this.cachedRegionLocations) {
         Map<byte[], HRegionLocation> tableLocations =
           getTableLocations(hri.getTableName());
@@ -1902,6 +1903,9 @@ public class HConnectionManager {
           rme.getHostname() + ":" + rme.getPort() + " according to " + source.getHostnamePort());
         updateCachedLocation(
             regionInfo, source, rme.getServerName(), rme.getLocationSeqNum());
+      } else if (RegionOpeningException.find(exception) != null) {
+        LOG.info("Region " + regionInfo.getRegionNameAsString() + " is being opened on "
+          + source.getHostnamePort() + "; not deleting the cache entry");
       } else {
         deleteCachedLocation(regionInfo, source);
       }
@@ -2163,13 +2167,11 @@ public class HConnectionManager {
           // Retry all actions in toReplay then clear it.
           if (!noRetry && !toReplay.isEmpty()) {
             if (isTraceEnabled) {
-              LOG.trace("Retrying due to errors: " + retriedErrors.getDescriptionAndClear());
+              LOG.trace("Retrying due to errors" + (lastRetry ? " (one last time)" : "")
+                   + ": " + retriedErrors.getDescriptionAndClear());
             }
             doRetry();
             if (lastRetry) {
-              if (isTraceEnabled) {
-                LOG.trace("No more retries");
-              }
               noRetry = true;
             }
           }
@@ -2201,7 +2203,7 @@ public class HConnectionManager {
           if (exceptions.isEmpty()) {
             return "";
           }
-          String result = makeException().getMessage();
+          String result = makeException().getExhaustiveDescription();
           exceptions.clear();
           actions.clear();
           addresses.clear();
@@ -2293,9 +2295,7 @@ public class HConnectionManager {
     int getNumberOfCachedRegionLocations(final byte[] tableName) {
       Integer key = Bytes.mapKey(tableName);
       synchronized (this.cachedRegionLocations) {
-        Map<byte[], HRegionLocation> tableLocs =
-          this.cachedRegionLocations.get(key);
-
+        Map<byte[], HRegionLocation> tableLocs = this.cachedRegionLocations.get(key);
         if (tableLocs == null) {
           return 0;
         }

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java Thu Mar 21 02:04:07 2013
@@ -121,18 +121,22 @@ extends RetriesExhaustedException {
 
   public String getExhaustiveDescription() {
     StringWriter errorWriter = new StringWriter();
+    PrintWriter pw = new PrintWriter(errorWriter);
     for (int i = 0; i < this.exceptions.size(); ++i) {
       Throwable t = this.exceptions.get(i);
       Row action = this.actions.get(i);
       String server = this.hostnameAndPort.get(i);
-      errorWriter.append("Error #" + i + " from [" + server + "] for ["
+      pw.append("Error");
+      if (this.exceptions.size() > 1) {
+        pw.append(" #" + i);
+      }
+      pw.append(" from [" + server + "] for ["
         + ((action == null) ? "unknown key" : Bytes.toStringBinary(action.getRow())) + "]");
       if (t != null) {
-        PrintWriter pw = new PrintWriter(errorWriter);
         t.printStackTrace(pw);
-        pw.flush();
       }
     }
+    pw.flush();
     return errorWriter.toString();
   }
 

Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionOpeningException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionOpeningException.java?rev=1459130&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionOpeningException.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionOpeningException.java Thu Mar 21 02:04:07 2013
@@ -0,0 +1,78 @@
+/*
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.exceptions;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.RemoteException;
+
+/**
+ * Subclass if the server knows the region is now on another server.
+ * This allows the client to call the new region server without calling the master.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RegionOpeningException extends NotServingRegionException {
+  private static final Log LOG = LogFactory.getLog(RegionOpeningException.class);
+  private static final long serialVersionUID = -7232903522310558395L;
+
+  public RegionOpeningException(String message) {
+    super(message);
+  }
+
+  /**
+   * Look for a RegionOpeningException in the exception:
+   *  - hadoop.ipc wrapped exceptions
+   *  - nested exceptions
+   * Returns null if we didn't find the exception.
+   * TODO: this code is mostly C/Ped from RegionMovedExecption. Due to the limitations of
+   *       generics it's not amenable to generalizing without adding parameters/isAssignableFrom.
+   *       Might make general if used in more places.
+   */
+  public static RegionOpeningException find(Object exception) {
+    if (exception == null || !(exception instanceof Throwable)) {
+      return null;
+    }
+    RegionOpeningException res = null;
+    Throwable cur = (Throwable)exception;
+    while (res == null && cur != null) {
+      if (cur instanceof RegionOpeningException) {
+        res = (RegionOpeningException) cur;
+      } else {
+        if (cur instanceof RemoteException) {
+          RemoteException re = (RemoteException) cur;
+          Exception e = re.unwrapRemoteException(RegionOpeningException.class);
+          if (e == null) {
+            e = re.unwrapRemoteException();
+          }
+          // unwrapRemoteException can return the exception given as a parameter when it cannot
+          //  unwrap it. In this case, there is no need to look further
+          // noinspection ObjectEquality
+          if (e != re) {
+            res = find(e);
+          }
+        }
+        cur = cur.getCause();
+      }
+    }
+    return res;
+  }
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Mar 21 02:04:07 2013
@@ -74,6 +74,7 @@ import org.apache.hadoop.hbase.exception
 import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
 import org.apache.hadoop.hbase.exceptions.RegionAlreadyInTransitionException;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
+import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
 import org.apache.hadoop.hbase.RegionServerStatusProtocol;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.ServerName;
@@ -2409,9 +2410,12 @@ public class HRegionServer implements Cl
       MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
       if (moveInfo != null) {
         throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
-      } else {
-        throw new NotServingRegionException("Region is not online: " + encodedRegionName);
       }
+      Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
+      if (isOpening != null && isOpening.booleanValue()) {
+        throw new RegionOpeningException("Region is being opened: " + encodedRegionName);
+      }
+      throw new NotServingRegionException("Region is not online: " + encodedRegionName);
     }
     return region;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java Thu Mar 21 02:04:07 2013
@@ -36,21 +36,17 @@ public class TestHRegionLocation {
   @Test
   public void testHashAndEqualsCode() {
     ServerName hsa1 = new ServerName("localhost", 1234, -1L);
-    HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-        hsa1, HConstants.NO_SEQNUM);
-    HRegionLocation hrl2 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-        hsa1, HConstants.NO_SEQNUM);
+    HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
+    HRegionLocation hrl2 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
     assertEquals(hrl1.hashCode(), hrl2.hashCode());
     assertTrue(hrl1.equals(hrl2));
-    HRegionLocation hrl3 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
-        hsa1, HConstants.NO_SEQNUM);
+    HRegionLocation hrl3 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa1);
     assertNotSame(hrl1, hrl3);
     // They are equal because they have same location even though they are
     // carrying different regions or timestamp.
     assertTrue(hrl1.equals(hrl3));
     ServerName hsa2 = new ServerName("localhost", 12345, -1L);
-    HRegionLocation hrl4 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO,
-        hsa2, HConstants.NO_SEQNUM);
+    HRegionLocation hrl4 = new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa2);
     // These have same HRI but different locations so should be different.
     assertFalse(hrl3.equals(hrl4));
     HRegionLocation hrl5 = new HRegionLocation(hrl4.getRegionInfo(),
@@ -61,8 +57,7 @@ public class TestHRegionLocation {
   @Test
   public void testToString() {
     ServerName hsa1 = new ServerName("localhost", 1234, -1L);
-    HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-      hsa1, HConstants.NO_SEQNUM);
+    HRegionLocation hrl1 = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, hsa1);
     System.out.println(hrl1.toString());
   }
 
@@ -70,10 +65,10 @@ public class TestHRegionLocation {
   public void testCompareTo() {
     ServerName hsa1 = new ServerName("localhost", 1234, -1L);
     HRegionLocation hsl1 =
-      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa1, HConstants.NO_SEQNUM);
+      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa1);
     ServerName hsa2 = new ServerName("localhost", 1235, -1L);
     HRegionLocation hsl2 =
-      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa2, HConstants.NO_SEQNUM);
+      new HRegionLocation(HRegionInfo.ROOT_REGIONINFO, hsa2);
     assertTrue(hsl1.compareTo(hsl1) == 0);
     assertTrue(hsl2.compareTo(hsl2) == 0);
     int compare1 = hsl1.compareTo(hsl2);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestCatalogTracker.java Thu Mar 21 02:04:07 2013
@@ -330,7 +330,7 @@ public class TestCatalogTracker {
     Mockito.doNothing().when(connection).close();
     // Make it so we return any old location when asked.
     final HRegionLocation anyLocation =
-      new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN, HConstants.NO_SEQNUM);
+      new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN);
     Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(anyLocation);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/catalog/TestMetaReaderEditorNoCluster.java Thu Mar 21 02:04:07 2013
@@ -178,7 +178,7 @@ public class TestMetaReaderEditorNoClust
       // Fix the location lookup so it 'works' though no network.  First
       // make an 'any location' object.
       final HRegionLocation anyLocation =
-        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn, HConstants.NO_SEQNUM);
+        new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, sn);
       // Return the any location object when locateRegion is called in HTable
       // constructor and when its called by ServerCallable (it uses getRegionLocation).
       // The ugly format below comes of 'Important gotcha on spying real objects!' from

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java Thu Mar 21 02:04:07 2013
@@ -99,7 +99,7 @@ public class HConnectionTestingUtility {
     HConnection c = HConnectionTestingUtility.getMockedConnection(conf);
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
-    final HRegionLocation loc = new HRegionLocation(hri, sn, HConstants.NO_SEQNUM);
+    final HRegionLocation loc = new HRegionLocation(hri, sn);
     Mockito.when(c.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Thu Mar 21 02:04:07 2013
@@ -424,7 +424,7 @@ public class TestHCM {
     assertNotNull(location);
 
     HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(), new ServerName(
-        location.getHostname(), location.getPort() - 1, 0L), HConstants.NO_SEQNUM);
+        location.getHostname(), location.getPort() - 1, 0L));
 
     // Same server as already in cache reporting - overwrites any value despite seqNum.
     int nextPort = location.getPort() + 1;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java Thu Mar 21 02:04:07 2013
@@ -271,7 +271,7 @@ public class TestLoadIncrementalHFilesSp
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
     final HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-        new ServerName("example.org", 1234, 0), HConstants.NO_SEQNUM);
+        new ServerName("example.org", 1234, 0));
     Mockito.when(c.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);

Modified: hbase/trunk/hbase-server/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/resources/log4j.properties?rev=1459130&r1=1459129&r2=1459130&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/resources/log4j.properties (original)
+++ hbase/trunk/hbase-server/src/test/resources/log4j.properties Thu Mar 21 02:04:07 2013
@@ -66,3 +66,5 @@ log4j.logger.org.apache.hadoop.hbase=DEB
 #See HBASE-4709
 log4j.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=ERROR
 log4j.org.apache.hadoop.metrics2.util.MBeans=ERROR
+# Enable this to get detailed connection error/retry logging.
+# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE