HBASE-21890 Use execute instead of submit to submit a task in RemoteProcedureDispatcher
Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
parent
e06beb0d72
commit
b3eb70c32d
|
@ -23,25 +23,21 @@ import java.lang.Thread.UncaughtExceptionHandler;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.DelayQueue;
|
import java.util.concurrent.DelayQueue;
|
||||||
import java.util.concurrent.Future;
|
|
||||||
import java.util.concurrent.FutureTask;
|
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.TimeUnit;
|
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.yetus.audience.InterfaceAudience;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
|
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
|
||||||
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedContainerWithTimestamp;
|
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedContainerWithTimestamp;
|
||||||
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
|
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
|
||||||
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
|
import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
|
||||||
|
|
||||||
|
@ -52,7 +48,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
|
||||||
* <li>Each server queue has a dispatch buffer</li>
|
* <li>Each server queue has a dispatch buffer</li>
|
||||||
* <li>Once the dispatch buffer reaches a threshold-size/time we send<li>
|
* <li>Once the dispatch buffer reaches a threshold-size/time we send<li>
|
||||||
* </ul>
|
* </ul>
|
||||||
* <p>Call {@link #start()} and then {@link #submitTask(Callable)}. When done,
|
* <p>Call {@link #start()} and then {@link #submitTask(Runnable)}. When done,
|
||||||
* call {@link #stop()}.
|
* call {@link #stop()}.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
|
@ -139,14 +135,7 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
|
protected abstract UncaughtExceptionHandler getUncaughtExceptionHandler();
|
||||||
return new UncaughtExceptionHandler() {
|
|
||||||
@Override
|
|
||||||
public void uncaughtException(Thread t, Throwable e) {
|
|
||||||
LOG.warn("Failed to execute remote procedures " + t.getName(), e);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
// ============================================================================================
|
// ============================================================================================
|
||||||
// Node Helpers
|
// Node Helpers
|
||||||
|
@ -197,14 +186,12 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
|
||||||
// ============================================================================================
|
// ============================================================================================
|
||||||
// Task Helpers
|
// Task Helpers
|
||||||
// ============================================================================================
|
// ============================================================================================
|
||||||
protected Future<Void> submitTask(Callable<Void> task) {
|
protected final void submitTask(Runnable task) {
|
||||||
return threadPool.submit(task);
|
threadPool.execute(task);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Future<Void> submitTask(Callable<Void> task, long delay, TimeUnit unit) {
|
protected final void submitTask(Runnable task, long delay, TimeUnit unit) {
|
||||||
final FutureTask<Void> futureTask = new FutureTask(task);
|
timeoutExecutor.add(new DelayedTask(task, delay, unit));
|
||||||
timeoutExecutor.add(new DelayedTask(futureTask, delay, unit));
|
|
||||||
return futureTask;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract void remoteDispatch(TRemote key, Set<RemoteProcedure> operations);
|
protected abstract void remoteDispatch(TRemote key, Set<RemoteProcedure> operations);
|
||||||
|
@ -254,19 +241,19 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Account of what procedures are running on remote node.
|
* Account of what procedures are running on remote node.
|
||||||
* @param <TEnv>
|
|
||||||
* @param <TRemote>
|
|
||||||
*/
|
*/
|
||||||
public interface RemoteNode<TEnv, TRemote> {
|
public interface RemoteNode<TEnv, TRemote> {
|
||||||
TRemote getKey();
|
TRemote getKey();
|
||||||
|
|
||||||
void add(RemoteProcedure<TEnv, TRemote> operation);
|
void add(RemoteProcedure<TEnv, TRemote> operation);
|
||||||
|
|
||||||
void dispatch();
|
void dispatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ArrayListMultimap<Class<?>, RemoteOperation> buildAndGroupRequestByType(final TEnv env,
|
protected ArrayListMultimap<Class<?>, RemoteOperation> buildAndGroupRequestByType(final TEnv env,
|
||||||
final TRemote remote, final Set<RemoteProcedure> remoteProcedures) {
|
final TRemote remote, final Set<RemoteProcedure> remoteProcedures) {
|
||||||
final ArrayListMultimap<Class<?>, RemoteOperation> requestByType = ArrayListMultimap.create();
|
final ArrayListMultimap<Class<?>, RemoteOperation> requestByType = ArrayListMultimap.create();
|
||||||
for (RemoteProcedure proc: remoteProcedures) {
|
for (RemoteProcedure proc : remoteProcedures) {
|
||||||
RemoteOperation operation = proc.remoteCallBuild(env, remote);
|
RemoteOperation operation = proc.remoteCallBuild(env, remote);
|
||||||
requestByType.put(operation.getClass(), operation);
|
requestByType.put(operation.getClass(), operation);
|
||||||
}
|
}
|
||||||
|
@ -297,9 +284,9 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (task instanceof DelayedTask) {
|
if (task instanceof DelayedTask) {
|
||||||
threadPool.execute(((DelayedTask)task).getObject());
|
threadPool.execute(((DelayedTask) task).getObject());
|
||||||
} else {
|
} else {
|
||||||
((BufferNode)task).dispatch();
|
((BufferNode) task).dispatch();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -390,10 +377,11 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delayed object that holds a FutureTask.
|
* Delayed object that holds a FutureTask.
|
||||||
|
* <p/>
|
||||||
* used to submit something later to the thread-pool.
|
* used to submit something later to the thread-pool.
|
||||||
*/
|
*/
|
||||||
private static final class DelayedTask extends DelayedContainerWithTimestamp<FutureTask<Void>> {
|
private static final class DelayedTask extends DelayedContainerWithTimestamp<Runnable> {
|
||||||
public DelayedTask(final FutureTask<Void> task, final long delay, final TimeUnit unit) {
|
public DelayedTask(Runnable task, long delay, TimeUnit unit) {
|
||||||
super(task, EnvironmentEdgeManager.currentTime() + unit.toMillis(delay));
|
super(task, EnvironmentEdgeManager.currentTime() + unit.toMillis(delay));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,141 @@
|
||||||
|
/**
|
||||||
|
* 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.procedure2;
|
||||||
|
|
||||||
|
import java.lang.Thread.UncaughtExceptionHandler;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Make sure the {@link UncaughtExceptionHandler} will be called when there are unchecked exceptions
|
||||||
|
* thrown in the task.
|
||||||
|
* <p/>
|
||||||
|
* See HBASE-21875 and HBASE-21890 for more details.
|
||||||
|
*/
|
||||||
|
@Category({ MasterTests.class, SmallTests.class })
|
||||||
|
public class TestRemoteProcedureDispatcherUncaughtExceptionHandler {
|
||||||
|
|
||||||
|
private static HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility();
|
||||||
|
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestRemoteProcedureDispatcherUncaughtExceptionHandler.class);
|
||||||
|
|
||||||
|
private static final class ExceptionHandler implements UncaughtExceptionHandler {
|
||||||
|
|
||||||
|
private Throwable error;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void uncaughtException(Thread t, Throwable e) {
|
||||||
|
this.error = e;
|
||||||
|
notifyAll();
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized void get() throws Throwable {
|
||||||
|
while (error == null) {
|
||||||
|
wait();
|
||||||
|
}
|
||||||
|
throw error;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final class Dispatcher extends RemoteProcedureDispatcher<Void, Integer> {
|
||||||
|
|
||||||
|
private final UncaughtExceptionHandler handler;
|
||||||
|
|
||||||
|
public Dispatcher(UncaughtExceptionHandler handler) {
|
||||||
|
super(UTIL.getConfiguration());
|
||||||
|
this.handler = handler;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
|
||||||
|
return handler;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void remoteDispatch(Integer key, Set<RemoteProcedure> operations) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void abortPendingOperations(Integer key, Set<RemoteProcedure> operations) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException thrown = ExpectedException.none();
|
||||||
|
|
||||||
|
private ExceptionHandler handler;
|
||||||
|
|
||||||
|
private Dispatcher dispatcher;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
handler = new ExceptionHandler();
|
||||||
|
dispatcher = new Dispatcher(handler);
|
||||||
|
dispatcher.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() {
|
||||||
|
dispatcher.stop();
|
||||||
|
dispatcher = null;
|
||||||
|
handler = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSubmit() throws Throwable {
|
||||||
|
String message = "inject error";
|
||||||
|
thrown.expect(RuntimeException.class);
|
||||||
|
thrown.expectMessage(message);
|
||||||
|
dispatcher.submitTask(new Runnable() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
throw new RuntimeException(message);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
handler.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDelayedSubmit() throws Throwable {
|
||||||
|
String message = "inject error";
|
||||||
|
thrown.expect(RuntimeException.class);
|
||||||
|
thrown.expectMessage(message);
|
||||||
|
dispatcher.submitTask(new Runnable() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
throw new RuntimeException(message);
|
||||||
|
}
|
||||||
|
}, 100, TimeUnit.MILLISECONDS);
|
||||||
|
handler.get();
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,9 +18,9 @@
|
||||||
package org.apache.hadoop.hbase.master.procedure;
|
package org.apache.hadoop.hbase.master.procedure;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.lang.Thread.UncaughtExceptionHandler;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
|
@ -81,6 +81,17 @@ public class RSProcedureDispatcher
|
||||||
RS_RPC_STARTUP_WAIT_TIME_CONF_KEY, DEFAULT_RS_RPC_STARTUP_WAIT_TIME);
|
RS_RPC_STARTUP_WAIT_TIME_CONF_KEY, DEFAULT_RS_RPC_STARTUP_WAIT_TIME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
|
||||||
|
return new UncaughtExceptionHandler() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void uncaughtException(Thread t, Throwable e) {
|
||||||
|
LOG.error("Unexpected error caught, this may cause the procedure to hang forever", e);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean start() {
|
public boolean start() {
|
||||||
if (!super.start()) {
|
if (!super.start()) {
|
||||||
|
@ -146,9 +157,7 @@ public class RSProcedureDispatcher
|
||||||
/**
|
/**
|
||||||
* Base remote call
|
* Base remote call
|
||||||
*/
|
*/
|
||||||
protected abstract class AbstractRSRemoteCall implements Callable<Void> {
|
protected abstract class AbstractRSRemoteCall implements Runnable {
|
||||||
@Override
|
|
||||||
public abstract Void call();
|
|
||||||
|
|
||||||
private final ServerName serverName;
|
private final ServerName serverName;
|
||||||
|
|
||||||
|
@ -279,10 +288,9 @@ public class RSProcedureDispatcher
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call() {
|
public void run() {
|
||||||
remoteCallFailed(procedureEnv,
|
remoteCallFailed(procedureEnv,
|
||||||
new RegionServerStoppedException("Server " + getServerName() + " is not online"));
|
new RegionServerStoppedException("Server " + getServerName() + " is not online"));
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -302,7 +310,7 @@ public class RSProcedureDispatcher
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call() {
|
public void run() {
|
||||||
request = ExecuteProceduresRequest.newBuilder();
|
request = ExecuteProceduresRequest.newBuilder();
|
||||||
if (LOG.isTraceEnabled()) {
|
if (LOG.isTraceEnabled()) {
|
||||||
LOG.trace("Building request with operations count=" + remoteProcedures.size());
|
LOG.trace("Building request with operations count=" + remoteProcedures.size());
|
||||||
|
@ -319,7 +327,6 @@ public class RSProcedureDispatcher
|
||||||
remoteCallFailed(procedureEnv, e);
|
remoteCallFailed(procedureEnv, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -387,7 +394,7 @@ public class RSProcedureDispatcher
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call() {
|
public void run() {
|
||||||
final OpenRegionRequest request =
|
final OpenRegionRequest request =
|
||||||
buildOpenRegionRequest(procedureEnv, getServerName(), operations);
|
buildOpenRegionRequest(procedureEnv, getServerName(), operations);
|
||||||
|
|
||||||
|
@ -401,7 +408,6 @@ public class RSProcedureDispatcher
|
||||||
remoteCallFailed(procedureEnv, e);
|
remoteCallFailed(procedureEnv, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private OpenRegionResponse sendRequest(final ServerName serverName,
|
private OpenRegionResponse sendRequest(final ServerName serverName,
|
||||||
|
@ -434,7 +440,7 @@ public class RSProcedureDispatcher
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call() {
|
public void run() {
|
||||||
final CloseRegionRequest request = operation.buildCloseRegionRequest(getServerName());
|
final CloseRegionRequest request = operation.buildCloseRegionRequest(getServerName());
|
||||||
try {
|
try {
|
||||||
CloseRegionResponse response = sendRequest(getServerName(), request);
|
CloseRegionResponse response = sendRequest(getServerName(), request);
|
||||||
|
@ -447,7 +453,6 @@ public class RSProcedureDispatcher
|
||||||
remoteCallFailed(procedureEnv, e);
|
remoteCallFailed(procedureEnv, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private CloseRegionResponse sendRequest(final ServerName serverName,
|
private CloseRegionResponse sendRequest(final ServerName serverName,
|
||||||
|
@ -473,7 +478,7 @@ public class RSProcedureDispatcher
|
||||||
* Compatibility class to open and close regions using old endpoints (openRegion/closeRegion) in
|
* Compatibility class to open and close regions using old endpoints (openRegion/closeRegion) in
|
||||||
* {@link AdminService}.
|
* {@link AdminService}.
|
||||||
*/
|
*/
|
||||||
protected class CompatRemoteProcedureResolver implements Callable<Void>, RemoteProcedureResolver {
|
protected class CompatRemoteProcedureResolver implements Runnable, RemoteProcedureResolver {
|
||||||
private final Set<RemoteProcedure> operations;
|
private final Set<RemoteProcedure> operations;
|
||||||
private final ServerName serverName;
|
private final ServerName serverName;
|
||||||
|
|
||||||
|
@ -484,9 +489,8 @@ public class RSProcedureDispatcher
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call() {
|
public void run() {
|
||||||
splitAndResolveOperation(serverName, operations, this);
|
splitAndResolveOperation(serverName, operations, this);
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
Loading…
Reference in New Issue