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:
Duo Zhang 2019-02-14 14:15:32 +08:00
parent e06beb0d72
commit b3eb70c32d
3 changed files with 177 additions and 44 deletions

View File

@ -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));
} }
} }

View File

@ -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();
}
}

View File

@ -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