You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2020/05/11 12:02:07 UTC

[hbase] branch master updated: HBASE-24341 The region should be removed from ConfigurationManager as a ConfigurationObserver when it is closed (#1675)

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

anoopsamjohn pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 68229c9  HBASE-24341 The region should be removed from ConfigurationManager as a ConfigurationObserver when it is closed (#1675)
68229c9 is described below

commit 68229c9c99b4548fb5850c9dba54850145218a42
Author: Joseph295 <51...@qq.com>
AuthorDate: Mon May 11 20:01:54 2020 +0800

    HBASE-24341 The region should be removed from ConfigurationManager as a ConfigurationObserver when it is closed (#1675)
    
    Co-authored-by: Joseph295 <jo...@gmail.com>
    Signed-off-by Anoop Sam John <an...@apache.org>
---
 .../hadoop/hbase/conf/ConfigurationManager.java    | 11 +++++
 .../hadoop/hbase/regionserver/HRegionServer.java   |  1 +
 .../TestRegionServerOnlineConfigChange.java        | 50 ++++++++++++++++++----
 3 files changed, 54 insertions(+), 8 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java
index 555a5c0..3e8d8fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationManager.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.conf;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -137,4 +138,14 @@ public class ConfigurationManager {
       return configurationObservers.size();
     }
   }
+
+  /**
+   * @return true if contains the observer, for unit test only
+   */
+  @VisibleForTesting
+  public boolean containsObserver(ConfigurationObserver observer) {
+    synchronized (configurationObservers) {
+      return configurationObservers.contains(observer);
+    }
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index d27162e..6828794 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -3305,6 +3305,7 @@ public class HRegionServer extends HasThread implements
       addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
     }
     this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
+    configurationManager.deregisterObserver(r);
     return toReturn != null;
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
index 157b29f..79c043e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
@@ -22,11 +22,16 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
@@ -34,6 +39,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -56,6 +62,7 @@ public class TestRegionServerOnlineConfigChange {
 
   private static final Logger LOG =
           LoggerFactory.getLogger(TestRegionServerOnlineConfigChange.class.getName());
+  private static final long WAIT_TIMEOUT = TimeUnit.MINUTES.toMillis(2);
   private static HBaseTestingUtility hbaseTestingUtility = new HBaseTestingUtility();
   private static Configuration conf = null;
 
@@ -71,24 +78,28 @@ public class TestRegionServerOnlineConfigChange {
 
 
   @BeforeClass
-  public static void setUp() throws Exception {
+  public static void setUpBeforeClass() throws Exception {
     conf = hbaseTestingUtility.getConfiguration();
-    hbaseTestingUtility.startMiniCluster();
+    hbaseTestingUtility.startMiniCluster(2);
     t1 = hbaseTestingUtility.createTable(TABLE1, COLUMN_FAMILY1);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    hbaseTestingUtility.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
     try (RegionLocator locator = hbaseTestingUtility.getConnection().getRegionLocator(TABLE1)) {
       RegionInfo firstHRI = locator.getAllRegionLocations().get(0).getRegion();
       r1name = firstHRI.getRegionName();
       rs1 = hbaseTestingUtility.getHBaseCluster().getRegionServer(
-          hbaseTestingUtility.getHBaseCluster().getServerWith(r1name));
+        hbaseTestingUtility.getHBaseCluster().getServerWith(r1name));
       r1 = rs1.getRegion(r1name);
     }
   }
 
-  @AfterClass
-  public static void tearDown() throws Exception {
-    hbaseTestingUtility.shutdownMiniCluster();
-  }
-
   /**
    * Check if the number of compaction threads changes online
    * @throws IOException
@@ -225,4 +236,27 @@ public class TestRegionServerOnlineConfigChange {
     assertEquals(newMajorCompactionJitter,
       hstore.getStoreEngine().getCompactionPolicy().getConf().getMajorCompactionJitter(), 0.00001);
   }
+
+  @Test
+  public void removeClosedRegionFromConfigurationManager() throws Exception {
+    try (Connection connection = ConnectionFactory.createConnection(conf)) {
+      Admin admin = connection.getAdmin();
+      assertTrue("The open region doesn't register as a ConfigurationObserver",
+        rs1.getConfigurationManager().containsObserver(r1));
+      admin.move(r1name);
+      hbaseTestingUtility.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+        @Override public boolean evaluate() throws Exception {
+          return rs1.getOnlineRegion(r1name) == null;
+        }
+      });
+      assertFalse("The closed region is not removed from ConfigurationManager",
+        rs1.getConfigurationManager().containsObserver(r1));
+      admin.move(r1name, rs1.getServerName());
+      hbaseTestingUtility.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
+        @Override public boolean evaluate() throws Exception {
+          return rs1.getOnlineRegion(r1name) != null;
+        }
+      });
+    }
+  }
 }