Skip to content

Commit

Permalink
HBASE-21890 Use execute instead of submit to submit a task in RemoteP…
Browse files Browse the repository at this point in the history
…rocedureDispatcher

Signed-off-by: Michael Stack <stack@apache.org>
  • Loading branch information
Apache9 committed Feb 14, 2019
1 parent e06beb0 commit b3eb70c
Show file tree
Hide file tree
Showing 3 changed files with 177 additions and 44 deletions.
Expand Up @@ -23,25 +23,21 @@
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.DelayQueue;
import java.util.concurrent.Future;
import java.util.concurrent.FutureTask;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;

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.DelayedContainerWithTimestamp;
import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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;

Expand All @@ -52,7 +48,7 @@
* <li>Each server queue has a dispatch buffer</li>
* <li>Once the dispatch buffer reaches a threshold-size/time we send<li>
* </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()}.
*/
@InterfaceAudience.Private
Expand Down Expand Up @@ -139,14 +135,7 @@ public void join() {
}
}

protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
return new UncaughtExceptionHandler() {
@Override
public void uncaughtException(Thread t, Throwable e) {
LOG.warn("Failed to execute remote procedures " + t.getName(), e);
}
};
}
protected abstract UncaughtExceptionHandler getUncaughtExceptionHandler();

// ============================================================================================
// Node Helpers
Expand Down Expand Up @@ -197,14 +186,12 @@ public boolean removeNode(final TRemote key) {
// ============================================================================================
// Task Helpers
// ============================================================================================
protected Future<Void> submitTask(Callable<Void> task) {
return threadPool.submit(task);
protected final void submitTask(Runnable task) {
threadPool.execute(task);
}

protected Future<Void> submitTask(Callable<Void> task, long delay, TimeUnit unit) {
final FutureTask<Void> futureTask = new FutureTask(task);
timeoutExecutor.add(new DelayedTask(futureTask, delay, unit));
return futureTask;
protected final void submitTask(Runnable task, long delay, TimeUnit unit) {
timeoutExecutor.add(new DelayedTask(task, delay, unit));
}

protected abstract void remoteDispatch(TRemote key, Set<RemoteProcedure> operations);
Expand Down Expand Up @@ -254,19 +241,19 @@ public interface RemoteProcedure<TEnv, TRemote> {

/**
* Account of what procedures are running on remote node.
* @param <TEnv>
* @param <TRemote>
*/
public interface RemoteNode<TEnv, TRemote> {
TRemote getKey();

void add(RemoteProcedure<TEnv, TRemote> operation);

void dispatch();
}

protected ArrayListMultimap<Class<?>, RemoteOperation> buildAndGroupRequestByType(final TEnv env,
final TRemote remote, final Set<RemoteProcedure> remoteProcedures) {
final ArrayListMultimap<Class<?>, RemoteOperation> requestByType = ArrayListMultimap.create();
for (RemoteProcedure proc: remoteProcedures) {
for (RemoteProcedure proc : remoteProcedures) {
RemoteOperation operation = proc.remoteCallBuild(env, remote);
requestByType.put(operation.getClass(), operation);
}
Expand Down Expand Up @@ -297,9 +284,9 @@ public void run() {
continue;
}
if (task instanceof DelayedTask) {
threadPool.execute(((DelayedTask)task).getObject());
threadPool.execute(((DelayedTask) task).getObject());
} else {
((BufferNode)task).dispatch();
((BufferNode) task).dispatch();
}
}
}
Expand Down Expand Up @@ -390,10 +377,11 @@ public String toString() {

/**
* Delayed object that holds a FutureTask.
* <p/>
* used to submit something later to the thread-pool.
*/
private static final class DelayedTask extends DelayedContainerWithTimestamp<FutureTask<Void>> {
public DelayedTask(final FutureTask<Void> task, final long delay, final TimeUnit unit) {
private static final class DelayedTask extends DelayedContainerWithTimestamp<Runnable> {
public DelayedTask(Runnable task, long delay, TimeUnit unit) {
super(task, EnvironmentEdgeManager.currentTime() + unit.toMillis(delay));
}
}
Expand Down
@@ -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();
}
}
Expand Up @@ -18,9 +18,9 @@
package org.apache.hadoop.hbase.master.procedure;

import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ServerName;
Expand Down Expand Up @@ -81,6 +81,17 @@ public RSProcedureDispatcher(final MasterServices master) {
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
public boolean start() {
if (!super.start()) {
Expand Down Expand Up @@ -146,9 +157,7 @@ public void serverRemoved(final ServerName serverName) {
/**
* Base remote call
*/
protected abstract class AbstractRSRemoteCall implements Callable<Void> {
@Override
public abstract Void call();
protected abstract class AbstractRSRemoteCall implements Runnable {

private final ServerName serverName;

Expand Down Expand Up @@ -279,10 +288,9 @@ public DeadRSRemoteCall(ServerName serverName, Set<RemoteProcedure> remoteProced
}

@Override
public Void call() {
public void run() {
remoteCallFailed(procedureEnv,
new RegionServerStoppedException("Server " + getServerName() + " is not online"));
return null;
}
}

Expand All @@ -302,7 +310,7 @@ public ExecuteProceduresRemoteCall(final ServerName serverName,
}

@Override
public Void call() {
public void run() {
request = ExecuteProceduresRequest.newBuilder();
if (LOG.isTraceEnabled()) {
LOG.trace("Building request with operations count=" + remoteProcedures.size());
Expand All @@ -319,7 +327,6 @@ public Void call() {
remoteCallFailed(procedureEnv, e);
}
}
return null;
}

@Override
Expand Down Expand Up @@ -387,7 +394,7 @@ public OpenRegionRemoteCall(final ServerName serverName,
}

@Override
public Void call() {
public void run() {
final OpenRegionRequest request =
buildOpenRegionRequest(procedureEnv, getServerName(), operations);

Expand All @@ -401,7 +408,6 @@ public Void call() {
remoteCallFailed(procedureEnv, e);
}
}
return null;
}

private OpenRegionResponse sendRequest(final ServerName serverName,
Expand Down Expand Up @@ -434,7 +440,7 @@ public CloseRegionRemoteCall(final ServerName serverName,
}

@Override
public Void call() {
public void run() {
final CloseRegionRequest request = operation.buildCloseRegionRequest(getServerName());
try {
CloseRegionResponse response = sendRequest(getServerName(), request);
Expand All @@ -447,7 +453,6 @@ public Void call() {
remoteCallFailed(procedureEnv, e);
}
}
return null;
}

private CloseRegionResponse sendRequest(final ServerName serverName,
Expand All @@ -473,7 +478,7 @@ private void remoteCallFailed(final MasterProcedureEnv env, final IOException e)
* Compatibility class to open and close regions using old endpoints (openRegion/closeRegion) in
* {@link AdminService}.
*/
protected class CompatRemoteProcedureResolver implements Callable<Void>, RemoteProcedureResolver {
protected class CompatRemoteProcedureResolver implements Runnable, RemoteProcedureResolver {
private final Set<RemoteProcedure> operations;
private final ServerName serverName;

Expand All @@ -484,9 +489,8 @@ public CompatRemoteProcedureResolver(final ServerName serverName,
}

@Override
public Void call() {
public void run() {
splitAndResolveOperation(serverName, operations, this);
return null;
}

@Override
Expand Down

0 comments on commit b3eb70c

Please sign in to comment.