You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/01/26 21:10:49 UTC

[hbase] branch master updated: HBASE-21775 The BufferedMutator doesn't ever refresh region location cache Fix server location metadata refresh in multi-actions

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new e92b62b  HBASE-21775 The BufferedMutator doesn't ever refresh region location cache Fix server location metadata refresh in multi-actions
e92b62b is described below

commit e92b62be547261e197c945bbb6d62be616e32e67
Author: Tommy Li <to...@microsoft.com>
AuthorDate: Thu Jan 24 15:37:49 2019 -0800

    HBASE-21775 The BufferedMutator doesn't ever refresh region location cache
    Fix server location metadata refresh in multi-actions
    
    Signed-off-by: stack <st...@apache.org>
---
 .../hbase/client/AsyncRequestFutureImpl.java       |  7 ++---
 .../hadoop/hbase/client/TestAsyncProcess.java      | 34 ++++++++++++++++++++++
 2 files changed, 37 insertions(+), 4 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index b75b0c6..525033d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
@@ -918,10 +918,9 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
   }
 
   private void cleanServerCache(ServerName server, Throwable regionException) {
-    if (tableName == null && ClientExceptionsUtil.isMetaClearingException(regionException)) {
-      // For multi-actions, we don't have a table name, but we want to make sure to clear the
-      // cache in case there were location-related exceptions. We don't to clear the cache
-      // for every possible exception that comes through, however.
+    if (ClientExceptionsUtil.isMetaClearingException(regionException)) {
+      // We want to make sure to clear the cache in case there were location-related exceptions.
+      // We don't to clear the cache for every possible exception that comes through, however.
       asyncProcess.connection.clearCaches(server);
     }
   }
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
index 025953f..ae30344 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.client.AsyncProcessTask.SubmittedRows;
 import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
+import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -1797,6 +1798,39 @@ public class TestAsyncProcess {
   }
 
   @Test
+  public void testRetryWithExceptionClearsMetaCache() throws Exception {
+    ClusterConnection conn = createHConnection();
+    Configuration myConf = conn.getConfiguration();
+    myConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
+
+    AsyncProcessWithFailure ap =
+        new AsyncProcessWithFailure(conn, myConf, new RegionOpeningException("test"));
+    BufferedMutatorParams bufferParam = createBufferedMutatorParams(ap, DUMMY_TABLE);
+    BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, bufferParam, ap);
+
+    Assert.assertNotNull(mutator.getAsyncProcess().createServerErrorTracker());
+
+    Assert.assertEquals(
+        conn.locateRegion(DUMMY_TABLE, DUMMY_BYTES_1, true, true).toString(),
+        new RegionLocations(loc1).toString());
+
+    Mockito.verify(conn, Mockito.times(0)).clearCaches(Mockito.any());
+
+    Put p = createPut(1, true);
+    mutator.mutate(p);
+
+    try {
+      mutator.flush();
+      Assert.fail();
+    } catch (RetriesExhaustedWithDetailsException expected) {
+      assertEquals(1, expected.getNumExceptions());
+      assertTrue(expected.getRow(0) == p);
+    }
+
+    Mockito.verify(conn, Mockito.times(1)).clearCaches(loc1.getServerName());
+  }
+
+  @Test
   public void testQueueRowAccess() throws Exception {
     ClusterConnection conn = createHConnection();
     BufferedMutatorImpl mutator = new BufferedMutatorImpl(conn, null, null,