You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/02/14 11:57:58 UTC

[hbase] branch branch-2.0 updated: HBASE-21900 Infinite loop in AsyncMetaRegionLocator if we can not get the location for meta

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

zhangduo pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.0 by this push:
     new f9a9348  HBASE-21900 Infinite loop in AsyncMetaRegionLocator if we can not get the location for meta
f9a9348 is described below

commit f9a9348cee0e6d7fb4abed1b675e9e5237b616bb
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Thu Feb 14 18:02:44 2019 +0800

    HBASE-21900 Infinite loop in AsyncMetaRegionLocator if we can not get the location for meta
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../hbase/client/AsyncMetaRegionLocator.java       |  1 +
 .../hadoop/hbase/zookeeper/ReadOnlyZKClient.java   | 11 +---
 .../client/TestAsyncMetaRegionLocatorFailFast.java | 67 ++++++++++++++++++++++
 .../org/apache/hadoop/hbase/util/FutureUtils.java  |  9 +++
 4 files changed, 80 insertions(+), 8 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
index ce3a2dd..f5b3f92 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
@@ -92,6 +92,7 @@ class AsyncMetaRegionLocator {
           metaRelocateFuture.set(null);
           future.complete(locs);
         });
+        return future;
       } else {
         CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
         if (future != null) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
index e92cca9..13448a5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
@@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
@@ -246,15 +247,9 @@ public final class ReadOnlyZKClient implements Closeable {
     }
   }
 
-  private static <T> CompletableFuture<T> failed(Throwable e) {
-    CompletableFuture<T> future = new CompletableFuture<>();
-    future.completeExceptionally(e);
-    return future;
-  }
-
   public CompletableFuture<byte[]> get(String path) {
     if (closed.get()) {
-      return failed(new DoNotRetryIOException("Client already closed"));
+      return FutureUtils.failedFuture(new DoNotRetryIOException("Client already closed"));
     }
     CompletableFuture<byte[]> future = new CompletableFuture<>();
     tasks.add(new ZKTask<byte[]>(path, future, "get") {
@@ -270,7 +265,7 @@ public final class ReadOnlyZKClient implements Closeable {
 
   public CompletableFuture<Stat> exists(String path) {
     if (closed.get()) {
-      return failed(new DoNotRetryIOException("Client already closed"));
+      return FutureUtils.failedFuture(new DoNotRetryIOException("Client already closed"));
     }
     CompletableFuture<Stat> future = new CompletableFuture<>();
     tasks.add(new ZKTask<Stat>(path, future, "exists") {
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
new file mode 100644
index 0000000..f29c3bf
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
@@ -0,0 +1,67 @@
+/**
+ * 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 java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.FutureUtils;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, SmallTests.class })
+public class TestAsyncMetaRegionLocatorFailFast {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncMetaRegionLocatorFailFast.class);
+
+  private static Configuration CONF = HBaseConfiguration.create();
+
+  private static AsyncMetaRegionLocator LOCATOR;
+
+  private static final class FaultyAsyncRegistry extends DoNothingAsyncRegistry {
+
+    public FaultyAsyncRegistry(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    public CompletableFuture<RegionLocations> getMetaRegionLocation() {
+      return FutureUtils.failedFuture(new DoNotRetryRegionException("inject error"));
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() {
+    LOCATOR = new AsyncMetaRegionLocator(new FaultyAsyncRegistry(CONF));
+  }
+
+  @Test(expected = DoNotRetryIOException.class)
+  public void test() throws IOException {
+    FutureUtils.get(LOCATOR.getRegionLocations(RegionInfo.DEFAULT_REPLICA_ID, false));
+  }
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
index 861dacb..6f0077c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FutureUtils.java
@@ -134,4 +134,13 @@ public final class FutureUtils {
       throw new IOException(cause);
     }
   }
+
+  /**
+   * Returns a CompletableFuture that is already completed exceptionally with the given exception.
+   */
+  public static <T> CompletableFuture<T> failedFuture(Throwable e) {
+    CompletableFuture<T> future = new CompletableFuture<>();
+    future.completeExceptionally(e);
+    return future;
+  }
 }
\ No newline at end of file