You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2015/12/15 20:49:14 UTC

[4/4] hbase git commit: HBASE-14968 ConcurrentModificationException in region close resulting in the region staying in closing state

HBASE-14968 ConcurrentModificationException in region close resulting in the region staying in closing state

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RegionReplicaFlushHandler.java


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6b8662c9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6b8662c9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6b8662c9

Branch: refs/heads/branch-1.0
Commit: 6b8662c9c268dc17eb988e5b70d819d9b8fbb105
Parents: 2b35976
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Dec 15 11:24:24 2015 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Dec 15 11:47:31 2015 -0800

----------------------------------------------------------------------
 .../hbase/coprocessor/CoprocessorHost.java      | 24 +++++++----
 .../hadoop/hbase/executor/EventHandler.java     |  7 +++-
 .../hbase/executor/TestExecutorService.java     | 43 +++++++++++++++++---
 3 files changed, 59 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b8662c9/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index ac909d6..b9c29fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -105,8 +105,11 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
    */
   private static Set<String> coprocessorNames =
       Collections.synchronizedSet(new HashSet<String>());
+
   public static Set<String> getLoadedCoprocessors() {
-      return coprocessorNames;
+    synchronized (coprocessorNames) {
+      return new HashSet(coprocessorNames);
+    }
   }
 
   /**
@@ -350,6 +353,7 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
    */
   static class EnvironmentPriorityComparator
       implements Comparator<CoprocessorEnvironment> {
+    @Override
     public int compare(final CoprocessorEnvironment env1,
         final CoprocessorEnvironment env2) {
       if (env1.getPriority() < env2.getPriority()) {
@@ -438,14 +442,16 @@ public abstract class CoprocessorHost<E extends CoprocessorEnvironment> {
         LOG.warn("Not stopping coprocessor "+impl.getClass().getName()+
             " because not active (state="+state.toString()+")");
       }
-      // clean up any table references
-      for (HTableInterface table: openTables) {
-        try {
-          ((HTableWrapper)table).internalClose();
-        } catch (IOException e) {
-          // nothing can be done here
-          LOG.warn("Failed to close " +
-              Bytes.toStringBinary(table.getTableName()), e);
+      synchronized (openTables) {
+        // clean up any table references
+        for (HTableInterface table: openTables) {
+          try {
+            ((HTableWrapper)table).internalClose();
+          } catch (IOException e) {
+            // nothing can be done here
+            LOG.warn("Failed to close " +
+                Bytes.toStringBinary(table.getTableName()), e);
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b8662c9/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
index 541089a..73fd7f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
@@ -121,6 +121,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
     return this;
   }
 
+  @Override
   public void run() {
     TraceScope chunk = Trace.startSpan(this.getClass().getSimpleName(), parent);
     try {
@@ -223,6 +224,10 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
    * @param t Throwable object
    */
   protected void handleException(Throwable t) {
-    LOG.error("Caught throwable while processing event " + eventType, t);
+    String msg = "Caught throwable while processing event " + eventType;
+    LOG.error(msg, t);
+    if (server != null) {
+      server.abort(msg, t);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b8662c9/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
index acb7ecf..25496ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hbase.executor;
 
 import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
-
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -29,7 +27,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.executor.ExecutorService.Executor;
 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -89,8 +89,8 @@ public class TestExecutorService {
     assertTrue(status.queuedEvents.isEmpty());
     assertEquals(5, status.running.size());
     checkStatusDump(status);
-    
-    
+
+
     // Now interrupt the running Executor
     synchronized (lock) {
       lock.set(false);
@@ -139,7 +139,7 @@ public class TestExecutorService {
     status.dumpTo(sw, "");
     String dump = sw.toString();
     LOG.info("Got status dump:\n" + dump);
-    
+
     assertTrue(dump.contains("Waiting on java.util.concurrent.atomic.AtomicBoolean"));
   }
 
@@ -172,5 +172,38 @@ public class TestExecutorService {
     }
   }
 
+  @Test
+  public void testAborting() throws Exception {
+    final Configuration conf = HBaseConfiguration.create();
+    final Server server = mock(Server.class);
+    when(server.getConfiguration()).thenReturn(conf);
+
+    ExecutorService executorService = new ExecutorService("unit_test");
+    executorService.startExecutorService(
+      ExecutorType.MASTER_SERVER_OPERATIONS, 1);
+
+
+    executorService.submit(new EventHandler(server, EventType.M_SERVER_SHUTDOWN) {
+      @Override
+      public void process() throws IOException {
+        throw new RuntimeException("Should cause abort");
+      }
+    });
+
+    Waiter.waitFor(conf, 30000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        try {
+          verify(server, times(1)).abort(anyString(), (Throwable) anyObject());
+          return true;
+        } catch (Throwable t) {
+          return false;
+        }
+      }
+    });
+
+    executorService.shutdown();
+  }
+
 }