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

zhangduo 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 3826e9ac0bc HBASE-29363 CompactSplit should not attempt to split 
secondary region replicas (#7048)
3826e9ac0bc is described below

commit 3826e9ac0bc45f77af03477e1f6d1281b01c90cf
Author: Charles Connell <[email protected]>
AuthorDate: Tue Jun 3 06:55:55 2025 -0400

    HBASE-29363 CompactSplit should not attempt to split secondary region 
replicas (#7048)
    
    Signed-off-by: Duo Zhang <[email protected]>
    (cherry picked from commit ae5400e77a951a69e4d80a8c7a4b54a5ad63a1e3)
---
 .../hadoop/hbase/regionserver/CompactSplit.java    | 15 ++++++++--
 .../hbase/regionserver/TestCompactSplitThread.java | 34 +++++++++++++++-------
 2 files changed, 36 insertions(+), 13 deletions(-)

diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
index 4593a7d2002..4ac1d7c6396 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.IntSupplier;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
@@ -205,7 +206,7 @@ public class CompactSplit implements CompactionRequester, 
PropagatingConfigurati
     // continuously growing, as well as the number of store files, see 
HBASE-26242.
     HRegion hr = (HRegion) r;
     try {
-      if (shouldSplitRegion() && hr.getCompactPriority() >= PRIORITY_USER) {
+      if (shouldSplitRegion(r.getRegionInfo()) && hr.getCompactPriority() >= 
PRIORITY_USER) {
         byte[] midKey = hr.checkSplit().orElse(null);
         if (midKey != null) {
           requestSplit(r, midKey);
@@ -503,12 +504,15 @@ public class CompactSplit implements CompactionRequester, 
PropagatingConfigurati
     return splits.getQueue().size();
   }
 
-  private boolean shouldSplitRegion() {
+  private boolean shouldSplitRegion(RegionInfo ri) {
     if (server.getNumberOfOnlineRegions() > 0.9 * regionSplitLimit) {
       LOG.warn("Total number of regions is approaching the upper limit " + 
regionSplitLimit + ". "
         + "Please consider taking a look at 
http://hbase.apache.org/book.html#ops.regionmgt";);
     }
-    return (regionSplitLimit > server.getNumberOfOnlineRegions());
+    return (regionSplitLimit > server.getNumberOfOnlineRegions()
+      // Do not attempt to split secondary region replicas, as this is not 
allowed and our request
+      // to do so will be rejected
+      && ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID);
   }
 
   /** Returns the regionSplitLimit */
@@ -807,6 +811,11 @@ public class CompactSplit implements CompactionRequester, 
PropagatingConfigurati
     return this.splits.getCorePoolSize();
   }
 
+  /** Exposed for unit testing */
+  long getSubmittedSplitsCount() {
+    return this.splits.getTaskCount();
+  }
+
   /**
    * {@inheritDoc}
    */
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
index d2f4e5108de..975b1fb631a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
@@ -30,14 +30,15 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -63,8 +64,8 @@ public class TestCompactSplitThread {
   /**
    * Setup the config for the cluster
    */
-  @BeforeClass
-  public static void setupCluster() throws Exception {
+  @Before
+  public void setupCluster() throws Exception {
     setupConf(TEST_UTIL.getConfiguration());
     TEST_UTIL.startMiniCluster(NUM_RS);
     fs = TEST_UTIL.getDFSCluster().getFileSystem();
@@ -91,12 +92,7 @@ public class TestCompactSplitThread {
   }
 
   @After
-  public void tearDown() throws Exception {
-    TEST_UTIL.deleteTable(tableName);
-  }
-
-  @AfterClass
-  public static void cleanupTest() throws Exception {
+  public void cleanupTest() throws Exception {
     try {
       TEST_UTIL.shutdownMiniCluster();
     } catch (Exception e) {
@@ -172,4 +168,22 @@ public class TestCompactSplitThread {
     Collection<String> hfiles = SnapshotTestingUtils.listHFileNames(fs, 
tableDir);
     assert (hfiles.size() > blockingStoreFiles + 1);
   }
+
+  @Test
+  public void testFlushWithRegionReplicas() throws Exception {
+    TableDescriptor htd =
+      
TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(2).build();
+    TEST_UTIL.createTable(htd, new byte[][] { family }, null);
+
+    // load the table
+    for (int i = 0; i < blockingStoreFiles + 1; i++) {
+      TEST_UTIL.loadTable(TEST_UTIL.getConnection().getTable(tableName), 
family);
+      TEST_UTIL.flush(tableName);
+    }
+
+    // One region split should have taken place, because the primary replica 
gets split, and not the
+    // secondary replica.
+    assertEquals(1, 
TEST_UTIL.getRSForFirstRegionInTable(tableName).getCompactSplitThread()
+      .getSubmittedSplitsCount());
+  }
 }

Reply via email to