You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/04/08 20:18:41 UTC

svn commit: r1585806 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop/hbase/ipc/thrift/ main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/h...

Author: liyin
Date: Tue Apr  8 18:18:41 2014
New Revision: 1585806

URL: http://svn.apache.org/r1585806
Log:
[HBASE-10905][0.89-fb] Be able to query a region location through the (Thrift)HRegionInterface

Author: gauravm

Summary:
In order to write a simplistic non-Java client (PHP in our case), where we don't want to push the region locating code to the client side, we can let the Region Server do it for us.

The client can pick up an Region Server in the regionservers tier, and get the location for any row in the table. The client is also able to request the server to disregard any previous cached location if it needs to.

Test Plan: TestGetRegionLocation and other unit tests

Reviewers: manukranthk, fan, rshroff, myang, daviddeng

Reviewed By: manukranthk

CC: hbase-eng@, andrewcox

Differential Revision: https://phabricator.fb.com/D1257376

Task ID: 4058577

Added:
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHRegionLocation.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetRegionLocation.java
Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java?rev=1585806&r1=1585805&r2=1585806&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java Tue Apr  8 18:18:41 2014
@@ -19,19 +19,20 @@
  */
 package org.apache.hadoop.hbase;
 
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+
 /**
  * Contains the HRegionInfo for the region and the HServerAddress for the
  * HRegionServer serving the region
  */
+@ThriftStruct
 public class HRegionLocation implements Comparable<HRegionLocation> {
   private HRegionInfo regionInfo;
   private HServerAddress serverAddress;
   private long serverStartCode;
 
-  public long getServerStartCode() {
-    return serverStartCode;
-  }
-
   /**
    * Constructor
    *
@@ -41,8 +42,12 @@ public class HRegionLocation implements 
   public HRegionLocation(HRegionInfo regionInfo, HServerAddress serverAddress) {
     this(regionInfo, serverAddress, -1);
   }
-  public HRegionLocation(HRegionInfo regionInfo, HServerAddress serverAddress,
-      long serverStartCode) {
+
+  @ThriftConstructor
+  public HRegionLocation(
+    @ThriftField(1) HRegionInfo regionInfo,
+    @ThriftField(2) HServerAddress serverAddress,
+    @ThriftField(3) long serverStartCode) {
     this.regionInfo = regionInfo;
     this.serverAddress = serverAddress;
     this.serverStartCode = serverStartCode;
@@ -86,15 +91,22 @@ public class HRegionLocation implements 
   }
 
   /** @return HRegionInfo */
+  @ThriftField(1)
   public HRegionInfo getRegionInfo(){
     return regionInfo;
   }
 
   /** @return HServerAddress */
+  @ThriftField(2)
   public HServerAddress getServerAddress(){
     return serverAddress;
   }
 
+  @ThriftField(3)
+  public long getServerStartCode() {
+    return serverStartCode;
+  }
+
   //
   // Comparable
   //

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java?rev=1585806&r1=1585805&r2=1585806&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java Tue Apr  8 18:18:41 2014
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.Restartable;
@@ -476,10 +477,23 @@ public interface HRegionInterface extend
   /**
    * Returns the list of buckets which represent the uniform depth histogram
    * for all the given regions
-   * @param regionName
+   * @param regionNames
    * @return
    * @throws IOException
    */
   public List<List<Bucket>> getHistograms(List<byte[]> regionNames)
       throws IOException;
+
+  /*
+   * Gets the location of the a particular row in a table.
+   *
+   * @param table
+   * @param row
+   * @param reload Should we reload the location cache? Set true if you get a
+   *               network exception / NotServingRegionException.
+   * @return
+   * @throws IOException
+   */
+  public HRegionLocation getLocation(byte[] table, byte[] row, boolean reload)
+      throws IOException;
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java?rev=1585806&r1=1585805&r2=1585806&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java Tue Apr  8 18:18:41 2014
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -660,4 +661,22 @@ public interface ThriftHRegionInterface 
       @ThriftException(type = ThriftHBaseException.class, id = 1) })
   public List<List<Bucket>> getHistograms(List<byte[]> regionNames)
       throws ThriftHBaseException;
+
+  /**
+   * Gets the location of the a particular row in a table.
+   *
+   * @param tableName
+   * @param row
+   * @param reload Should we reload the location cache? Set true if you get a
+   *               network exception / NotServingRegionException.
+   * @return
+   * @throws ThriftHBaseException
+   */
+  @ThriftMethod(value = "getLocation", exception = {
+    @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  public HRegionLocation getLocation(
+    @ThriftField(name="tableName") byte[] tableName,
+    @ThriftField(name="row") byte[] row,
+    @ThriftField(name="reload") boolean reload)
+      throws ThriftHBaseException;
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java?rev=1585806&r1=1585805&r2=1585806&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java Tue Apr  8 18:18:41 2014
@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.client.Delete;
@@ -1208,7 +1209,7 @@ public class HBaseToThriftAdapter implem
 
   @Override
   public List<List<Bucket>> getHistograms(List<byte[]> regionNames)
-      throws IOException {
+    throws IOException {
     preProcess();
     try {
       return connection.getHistograms(regionNames);
@@ -1225,4 +1226,23 @@ public class HBaseToThriftAdapter implem
     }
   }
 
+  @Override
+  public HRegionLocation getLocation(byte[] table, byte[] row, boolean reload)
+    throws IOException {
+    preProcess();
+    try {
+      return connection.getLocation(table, row, reload);
+    } catch (ThriftHBaseException te) {
+      Exception e = te.getServerJavaException();
+      handleIOException(e);
+      LOG.warn("Unexpected Exception: " + e);
+      throw new RuntimeException(e);
+    } catch (Exception e) {
+      refreshConnectionAndThrowIOException(e);
+      return null;
+    } finally {
+      postProcess();
+    }
+  }
+
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1585806&r1=1585805&r2=1585806&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue Apr  8 18:18:41 2014
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.UnknownSc
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.MultiAction;
 import org.apache.hadoop.hbase.client.MultiPut;
 import org.apache.hadoop.hbase.client.MultiPutResponse;
@@ -109,6 +110,7 @@ import org.apache.hadoop.hbase.client.Ro
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.ServerConnection;
 import org.apache.hadoop.hbase.client.ServerConnectionManager;
+import org.apache.hadoop.hbase.client.TableServers;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -150,6 +152,7 @@ import org.apache.hadoop.hbase.util.Para
 import org.apache.hadoop.hbase.util.ParamFormatter;
 import org.apache.hadoop.hbase.util.RuntimeHaltAbortStrategy;
 import org.apache.hadoop.hbase.util.Sleeper;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -230,6 +233,8 @@ public class HRegionServer implements HR
   protected final Configuration conf;
 
   private final ServerConnection connection;
+  private final TableServers regionServerConnection;
+
   protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
   private FileSystem fs;
   private Path rootDir;
@@ -445,6 +450,13 @@ public class HRegionServer implements HR
     this.fsOk = true;
     this.conf = conf;
     this.connection = ServerConnectionManager.getConnection(conf);
+    if (connection instanceof TableServers) {
+      // This is always true in the current case...
+      regionServerConnection = (TableServers) connection;
+    } else {
+      // ... but handling this possibility, just in case something changes.
+      regionServerConnection = new TableServers(conf);
+    }
 
     this.isOnline = false;
 
@@ -4103,4 +4115,13 @@ public class HRegionServer implements HR
       throws IOException {
     throw new NotImplementedException("HRegionserver.callEndpoint");
   }
+
+  @Override
+  public HRegionLocation getLocation(byte[] table, byte[] row, boolean reload)
+    throws IOException {
+    if (reload) {
+      return regionServerConnection.relocateRegion(new StringBytes(table), row);
+    }
+    return regionServerConnection.locateRegion(new StringBytes(table), row);
+  }
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java?rev=1585806&r1=1585805&r2=1585806&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java Tue Apr  8 18:18:41 2014
@@ -32,6 +32,7 @@ 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.HRegionLocation;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -645,9 +646,18 @@ public class ThriftHRegionServer impleme
   }
 
   @Override
-  @ThriftMethod(value = "getHistograms", exception = @ThriftException(type = ThriftHBaseException.class, id = 1))
   public List<List<Bucket>> getHistograms(List<byte[]> regionNames)
-      throws ThriftHBaseException {
+    throws ThriftHBaseException {
     return this.getHistograms(regionNames);
   }
+
+  @Override
+  public HRegionLocation getLocation(byte[] table, byte[] row, boolean reload)
+    throws ThriftHBaseException {
+    try {
+      return server.getLocation(table, row, reload);
+    } catch (IOException e) {
+      throw new ThriftHBaseException(e);
+    }
+  }
 }

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHRegionLocation.java?rev=1585806&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHRegionLocation.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHRegionLocation.java Tue Apr  8 18:18:41 2014
@@ -0,0 +1,58 @@
+/**
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.client;
+
+import junit.framework.Assert;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+
+/**
+ * This tests the serialization and deserialization of HRegionLocation.
+ */
+public class TestHRegionLocation {
+  @Test
+  public void testSwiftSerDe() throws Exception {
+    HRegionLocation location = createDummyHRegionLocation();
+    byte[] serializedLocation = Bytes.writeThriftBytes(location,
+      HRegionLocation.class);
+
+    HRegionLocation locationCopy = Bytes.readThriftBytes(serializedLocation,
+      HRegionLocation.class);
+
+    Assert.assertEquals("HRegionLocation did not serialize correctly",
+      location, locationCopy);
+  }
+
+
+  private static HRegionLocation createDummyHRegionLocation() {
+    HRegionInfo hri = new HRegionInfo(
+      new HTableDescriptor(Bytes.toBytes("table")),
+      Bytes.toBytes("aaa"),
+      Bytes.toBytes("zzz"),
+      false,
+      123456);
+    return new HRegionLocation(hri, new HServerAddress("localhost", 1234));
+  }
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetRegionLocation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetRegionLocation.java?rev=1585806&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetRegionLocation.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetRegionLocation.java Tue Apr  8 18:18:41 2014
@@ -0,0 +1,196 @@
+/**
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.regionserver;
+
+import junit.framework.Assert;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HServerInfo;
+import org.apache.hadoop.hbase.client.TableServers;
+import org.apache.hadoop.hbase.ipc.HRegionInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestGetRegionLocation {
+  private static final Log LOG = LogFactory.getLog(TestGetRegionLocation.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static Configuration CONF = TEST_UTIL.getConfiguration();
+  private final static byte[] TABLE = Bytes.toBytes("table");
+  private final static byte[] FAMILY = Bytes.toBytes("family");
+  private final static byte[][] FAMILIES = { FAMILY };
+  private final static byte[] START_KEY = Bytes.toBytes("aaa");
+  private final static byte[] END_KEY = Bytes.toBytes("zzz");
+  private final static int NUM_REGIONS = 10;
+  private final static int NUM_SLAVES = 4;
+
+  @Before
+  public void setUp() throws IOException, InterruptedException {
+    TEST_UTIL.startMiniCluster(NUM_SLAVES);
+    TEST_UTIL.createTable(TABLE, FAMILIES, 1, START_KEY, END_KEY, NUM_REGIONS);
+  }
+
+  @After
+  public void tearDown() throws IOException, InterruptedException {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  /**
+   * This test tests the getRegionLocation() call in the
+   * (Thrift)HRegionInterface API.
+   *
+   * @throws InterruptedException
+   * @throws IOException
+   */
+  @Test
+  public void testGetRegionLocation() throws InterruptedException, IOException {
+    List<HRegionServer> regionServers = TEST_UTIL.getOnlineRegionServers();
+    List<HServerInfo> serverInfos = new ArrayList<>();
+    Map<HRegionInfo, HServerInfo> regionInfoToServerInfoMap = new HashMap<>();
+
+    // Wait till all the regions becomes assigned.
+    TEST_UTIL.waitForOnlineRegionsToBeAssigned(NUM_REGIONS);
+
+    // Get the list of servers, and regions, and regions->servers mapping.
+    for (HRegionServer regionServer : regionServers) {
+      serverInfos.add(regionServer.getServerInfo());
+      for (HRegion or : regionServer.getOnlineRegions()) {
+        regionInfoToServerInfoMap.put(or.getRegionInfo(),
+          regionServer.getServerInfo());
+      }
+    }
+
+    TableServers connection = new TableServers(CONF);
+
+    // Iterate through each of the servers, and check if the locations for
+    // all the regions are correct.
+    for (HServerInfo serverInfo : serverInfos) {
+      HRegionInterface server =
+        connection.getHRegionConnection(serverInfo.getServerAddress());
+
+      for (Map.Entry<HRegionInfo, HServerInfo> entry :
+        regionInfoToServerInfoMap.entrySet()) {
+        HRegionInfo info = entry.getKey();
+
+        // Get the location for this particular region's start key...
+        HRegionLocation location =
+          server.getLocation(info.getTableDesc().getName(),
+            info.getStartKey(),
+            false);
+
+        // ... which should be the same as the actual location of this region.
+        Assert.assertEquals(
+          "getLocation() returned an incorrect server location for region: " +
+            info.getRegionNameAsString(),
+          entry.getValue().getServerAddress(),
+          location.getServerAddress());
+      }
+    }
+
+    // Now let us try moving a random region (let's pick the first)
+    // to a different location, and see if the getLocation works with reloading
+    Map.Entry<HRegionInfo, HServerInfo> firstRegion =
+      regionInfoToServerInfoMap.entrySet().iterator().next();
+    HRegionInfo firstRegionInfo = firstRegion.getKey();
+    HServerInfo firstServerInfo = firstRegion.getValue();
+
+    LOG.info("Region: " + firstRegionInfo.getRegionNameAsString() +
+      " was located at " + firstServerInfo.getServerAddress().toString());
+
+    // Pick up a server to query getRegionLocation
+    HRegionInterface server =
+      connection.getHRegionConnection(
+        serverInfos.iterator().next().getServerAddress());
+
+    // Check that the location before moving is sane
+    HRegionLocation locationBeforeMoving =
+      server.getLocation(firstRegionInfo.getTableDesc().getName(),
+        firstRegionInfo.getStartKey(), false);
+
+    Assert.assertEquals(
+      "getLocation() returned an incorrect server location for region: " +
+        firstRegionInfo.getRegionNameAsString(),
+        firstServerInfo.getServerAddress(),
+        locationBeforeMoving.getServerAddress()
+    );
+
+    HServerInfo targetServer = null;
+    // Find a targetServer to host the first region
+    for (HServerInfo serverInfo : serverInfos) {
+      if (serverInfo.equals(firstServerInfo)) {
+        continue;
+      }
+      targetServer = serverInfo;
+    }
+
+    LOG.info("Region: " + firstRegionInfo.getRegionNameAsString() +
+      " will be moved from: " + firstServerInfo.getServerAddress().toString() +
+      " to: " + targetServer.getServerAddress().toString());
+
+    // Now move the region to the target server
+    TEST_UTIL.getHBaseAdmin().moveRegion(firstRegionInfo.getRegionName(),
+      targetServer.getServerAddress().toString());
+
+    // Wait till the region becomes assigned.
+    TEST_UTIL.waitForOnlineRegionsToBeAssigned(NUM_REGIONS);
+
+    HRegionLocation staleLocationAfterMoving =
+      server.getLocation(firstRegionInfo.getTableDesc().getName(),
+        firstRegionInfo.getStartKey(), false);
+
+    LOG.info("As per (stale) cache, region: " +
+      firstRegionInfo.getRegionNameAsString() +
+      " was located at " +
+      staleLocationAfterMoving.getServerAddress().toString());
+
+    // Getting the location after reloading the cache.
+    HRegionLocation newLocationAfterMoving =
+      server.getLocation(firstRegionInfo.getTableDesc().getName(),
+      firstRegionInfo.getStartKey(), true);
+
+    LOG.info("As per (fresh) cache, region: " +
+      firstRegionInfo.getRegionNameAsString() +
+      " was located at " +
+      newLocationAfterMoving.getServerAddress().toString());
+
+    // The new location after reloading the cache, should be the same as
+    // what we expect.
+    Assert.assertEquals(
+      "getLocation() returned a stale server location for region: " +
+        firstRegionInfo.getRegionNameAsString(),
+      targetServer.getServerAddress(),
+      newLocationAfterMoving.getServerAddress()
+    );
+
+  }
+
+}