HBASE-21900 Infinite loop in AsyncMetaRegionLocator if we can not get the location for meta
Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
parent
b3eb70c32d
commit
8bed65fa01
|
@ -92,6 +92,7 @@ class AsyncMetaRegionLocator {
|
||||||
metaRelocateFuture.set(null);
|
metaRelocateFuture.set(null);
|
||||||
future.complete(locs);
|
future.complete(locs);
|
||||||
});
|
});
|
||||||
|
return future;
|
||||||
} else {
|
} else {
|
||||||
CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
|
CompletableFuture<RegionLocations> future = metaRelocateFuture.get();
|
||||||
if (future != null) {
|
if (future != null) {
|
||||||
|
|
|
@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||||
|
import org.apache.hadoop.hbase.util.FutureUtils;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
@ -252,15 +253,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) {
|
public CompletableFuture<byte[]> get(String path) {
|
||||||
if (closed.get()) {
|
if (closed.get()) {
|
||||||
return failed(new DoNotRetryIOException("Client already closed"));
|
return FutureUtils.failedFuture(new DoNotRetryIOException("Client already closed"));
|
||||||
}
|
}
|
||||||
CompletableFuture<byte[]> future = new CompletableFuture<>();
|
CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||||
tasks.add(new ZKTask<byte[]>(path, future, "get") {
|
tasks.add(new ZKTask<byte[]>(path, future, "get") {
|
||||||
|
@ -276,7 +271,7 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
|
|
||||||
public CompletableFuture<Stat> exists(String path) {
|
public CompletableFuture<Stat> exists(String path) {
|
||||||
if (closed.get()) {
|
if (closed.get()) {
|
||||||
return failed(new DoNotRetryIOException("Client already closed"));
|
return FutureUtils.failedFuture(new DoNotRetryIOException("Client already closed"));
|
||||||
}
|
}
|
||||||
CompletableFuture<Stat> future = new CompletableFuture<>();
|
CompletableFuture<Stat> future = new CompletableFuture<>();
|
||||||
tasks.add(new ZKTask<Stat>(path, future, "exists") {
|
tasks.add(new ZKTask<Stat>(path, future, "exists") {
|
||||||
|
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
|
@ -134,4 +134,13 @@ public final class FutureUtils {
|
||||||
throw new IOException(cause);
|
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;
|
||||||
|
}
|
||||||
}
|
}
|
Loading…
Reference in New Issue