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

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


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 8756ef6  HBASE-21890 Use execute instead of submit to submit a task in 
RemoteProcedureDispatcher
8756ef6 is described below

commit 8756ef6c5360f43d981058e06e0672b8a454fcca
Author: Duo Zhang <zhang...@apache.org>
AuthorDate: Thu Feb 14 14:15:32 2019 +0800

    HBASE-21890 Use execute instead of submit to submit a task in 
RemoteProcedureDispatcher
    
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../procedure2/RemoteProcedureDispatcher.java      |  46 +++----
 ...rocedureDispatcherUncaughtExceptionHandler.java | 141 +++++++++++++++++++++
 .../master/procedure/RSProcedureDispatcher.java    |  34 ++---
 3 files changed, 177 insertions(+), 44 deletions(-)

diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index 9c1f5ca..4a65796 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -23,25 +23,21 @@ import java.lang.Thread.UncaughtExceptionHandler;
 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;
 
@@ -52,7 +48,7 @@ import 
org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
  * <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
@@ -139,14 +135,7 @@ public abstract class RemoteProcedureDispatcher<TEnv, 
TRemote extends Comparable
     }
   }
 
-  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
@@ -197,14 +186,12 @@ public abstract class RemoteProcedureDispatcher<TEnv, 
TRemote extends Comparable
   // 
============================================================================================
   //  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);
@@ -254,19 +241,19 @@ public abstract class RemoteProcedureDispatcher<TEnv, 
TRemote extends Comparable
 
   /**
    * 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);
     }
@@ -297,9 +284,9 @@ public abstract class RemoteProcedureDispatcher<TEnv, 
TRemote extends Comparable
           continue;
         }
         if (task instanceof DelayedTask) {
-          threadPool.execute(((DelayedTask)task).getObject());
+          threadPool.execute(((DelayedTask) task).getObject());
         } 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.
+   * <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));
     }
   };
diff --git 
a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestRemoteProcedureDispatcherUncaughtExceptionHandler.java
 
b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestRemoteProcedureDispatcherUncaughtExceptionHandler.java
new file mode 100644
index 0000000..7f44fc3
--- /dev/null
+++ 
b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestRemoteProcedureDispatcherUncaughtExceptionHandler.java
@@ -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();
+  }
+}
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 638f9d3..0b552f3 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -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;
@@ -82,6 +82,17 @@ public class RSProcedureDispatcher
   }
 
   @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()) {
       return false;
@@ -146,9 +157,7 @@ public class RSProcedureDispatcher
   /**
    * 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;
 
@@ -279,10 +288,9 @@ public class RSProcedureDispatcher
     }
 
     @Override
-    public Void call() {
+    public void run() {
       remoteCallFailed(procedureEnv,
         new RegionServerStoppedException("Server " + getServerName() + " is 
not online"));
-      return null;
     }
   }
 
@@ -302,7 +310,7 @@ public class RSProcedureDispatcher
     }
 
     @Override
-    public Void call() {
+    public void run() {
       request = ExecuteProceduresRequest.newBuilder();
       if (LOG.isTraceEnabled()) {
         LOG.trace("Building request with operations count=" + 
remoteProcedures.size());
@@ -319,7 +327,6 @@ public class RSProcedureDispatcher
           remoteCallFailed(procedureEnv, e);
         }
       }
-      return null;
     }
 
     @Override
@@ -387,7 +394,7 @@ public class RSProcedureDispatcher
     }
 
     @Override
-    public Void call() {
+    public void run() {
       final OpenRegionRequest request =
           buildOpenRegionRequest(procedureEnv, getServerName(), operations);
 
@@ -401,7 +408,6 @@ public class RSProcedureDispatcher
           remoteCallFailed(procedureEnv, e);
         }
       }
-      return null;
     }
 
     private OpenRegionResponse sendRequest(final ServerName serverName,
@@ -434,7 +440,7 @@ public class RSProcedureDispatcher
     }
 
     @Override
-    public Void call() {
+    public void run() {
       final CloseRegionRequest request = 
operation.buildCloseRegionRequest(getServerName());
       try {
         CloseRegionResponse response = sendRequest(getServerName(), request);
@@ -447,7 +453,6 @@ public class RSProcedureDispatcher
           remoteCallFailed(procedureEnv, e);
         }
       }
-      return null;
     }
 
     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
    * {@link AdminService}.
    */
-  protected class CompatRemoteProcedureResolver implements Callable<Void>, 
RemoteProcedureResolver {
+  protected class CompatRemoteProcedureResolver implements Runnable, 
RemoteProcedureResolver {
     private final Set<RemoteProcedure> operations;
     private final ServerName serverName;
 
@@ -484,9 +489,8 @@ public class RSProcedureDispatcher
     }
 
     @Override
-    public Void call() {
+    public void run() {
       splitAndResolveOperation(serverName, operations, this);
-      return null;
     }
 
     @Override

Reply via email to