You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/05/17 23:53:37 UTC

[hbase] branch branch-2 updated: Revert "HBASE-22169 Open region failed cause memory leak"

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 64f5857  Revert "HBASE-22169 Open region failed cause memory leak"
64f5857 is described below

commit 64f5857c2dc769eda5a9cafe36a2f2166e266797
Author: stack <st...@apache.org>
AuthorDate: Fri May 17 16:53:14 2019 -0700

    Revert "HBASE-22169 Open region failed cause memory leak"
    
    This reverts commit 006170654beec73270efbf1c9d0cbb997d105bbc.
---
 .../apache/hadoop/hbase/regionserver/HRegion.java  | 38 ++++++++-----------
 .../hadoop/hbase/regionserver/TestHRegion.java     | 44 ----------------------
 .../TestOpenSeqNumUnexpectedIncrease.java          |  9 -----
 3 files changed, 15 insertions(+), 76 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 8566b59..24bafab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7283,29 +7283,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   protected HRegion openHRegion(final CancelableProgressable reporter)
   throws IOException {
-    try {
-      // Refuse to open the region if we are missing local compression support
-      checkCompressionCodecs();
-      // Refuse to open the region if encryption configuration is incorrect or
-      // codec support is missing
-      checkEncryption();
-      // Refuse to open the region if a required class cannot be loaded
-      checkClassLoading();
-      this.openSeqNum = initialize(reporter);
-      this.mvcc.advanceTo(openSeqNum);
-      // The openSeqNum must be increased every time when a region is assigned, as we rely on it to
-      // determine whether a region has been successfully reopened. So here we always write open
-      // marker, even if the table is read only.
-      if (wal != null && getRegionServerServices() != null &&
-        RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
-        writeRegionOpenMarker(wal, openSeqNum);
-      }
-    } catch(Throwable t) {
-      // By coprocessor path wrong region will open failed,
-      // MetricsRegionWrapperImpl is already init and not close,
-      // add region close when open failed
-      this.close();
-      throw t;
+    // Refuse to open the region if we are missing local compression support
+    checkCompressionCodecs();
+    // Refuse to open the region if encryption configuration is incorrect or
+    // codec support is missing
+    checkEncryption();
+    // Refuse to open the region if a required class cannot be loaded
+    checkClassLoading();
+    this.openSeqNum = initialize(reporter);
+    this.mvcc.advanceTo(openSeqNum);
+    // The openSeqNum must be increased every time when a region is assigned, as we rely on it to
+    // determine whether a region has been successfully reopened. So here we always write open
+    // marker, even if the table is read only.
+    if (wal != null && getRegionServerServices() != null &&
+      RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
+      writeRegionOpenMarker(wal, openSeqNum);
     }
     return this;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 2e3747e..c09f702 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -41,7 +41,6 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
@@ -53,14 +52,11 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Objects;
 import java.util.TreeMap;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -166,7 +162,6 @@ import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hbase.wal.WALSplitter;
-import org.apache.hadoop.metrics2.MetricsExecutor;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -6283,45 +6278,6 @@ public class TestHRegion {
         getCoprocessors().contains(ReplicationObserver.class.getSimpleName()));
   }
 
-  // make sure region is success close when coprocessor wrong region open failed
-  @Test
-  public void testOpenRegionFailedMemoryLeak() throws Exception {
-    final ServerName serverName = ServerName.valueOf("testOpenRegionFailed", 100, 42);
-    final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
-
-    HTableDescriptor htd
-      = new HTableDescriptor(TableName.valueOf("testOpenRegionFailed"));
-    htd.addFamily(new HColumnDescriptor(fam1));
-    htd.setValue("COPROCESSOR$1", "hdfs://test/test.jar|test||");
-
-    HRegionInfo hri = new HRegionInfo(htd.getTableName(),
-      HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
-    ScheduledExecutorService executor = CompatibilitySingletonFactory.getInstance(
-      MetricsExecutor.class).getExecutor();
-    for (int i = 0; i < 20 ; i++) {
-      try {
-        HRegion.openHRegion(hri, htd, rss.getWAL(hri),
-          TEST_UTIL.getConfiguration(), rss, null);
-      }catch(Throwable t){
-        LOG.info("Expected exception, continue");
-      }
-    }
-    TimeUnit.SECONDS.sleep(MetricsRegionWrapperImpl.PERIOD);
-    Field[] fields = ThreadPoolExecutor.class.getDeclaredFields();
-    boolean found = false;
-    for(Field field : fields){
-      if(field.getName().equals("workQueue")){
-        field.setAccessible(true);
-        BlockingQueue<Runnable> workQueue = (BlockingQueue<Runnable>)field.get(executor);
-        //there are still two task not cancel, can not cause to memory lack
-        Assert.assertTrue("ScheduledExecutor#workQueue should equals 2 , please check region is " +
-          "close", 2 == workQueue.size());
-        found = true;
-      }
-    }
-    Assert.assertTrue("can not find workQueue, test failed", found);
-  }
-
   /**
    * The same as HRegion class, the only difference is that instantiateHStore will
    * create a different HStore - HStoreForTesting. [HBASE-8518]
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java
index e013f62..11db694 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -76,13 +74,6 @@ public class TestOpenSeqNumUnexpectedIncrease {
         throw new IOException("Inject error for testing");
       }
     }
-
-    public Map<byte[], List<HStoreFile>> close() throws IOException {
-      //skip close
-      return null;
-    }
-
-
   }
 
   @BeforeClass