Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
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;
Expand Down Expand Up @@ -90,6 +91,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
private volatile ThreadPoolExecutor longCompactions;
private volatile ThreadPoolExecutor shortCompactions;
private volatile ThreadPoolExecutor splits;
// Used in unit testing
private int splitCounter;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to not introduce field in normal code which is only used for testing?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, done


private volatile ThroughputController compactionThroughputController;
private volatile Set<String> underCompactionStores = ConcurrentHashMap.newKeySet();
Expand All @@ -112,6 +115,8 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
// compaction throughput controller
this.compactionThroughputController =
CompactionThroughputControllerFactory.create(server, conf);

this.splitCounter = 0;
}

// only for test
Expand Down Expand Up @@ -205,7 +210,7 @@ public synchronized boolean requestSplit(final Region r) {
// 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);
Expand Down Expand Up @@ -235,6 +240,7 @@ private synchronized void requestSplit(final Region r, byte[] midKey, User user)
}
try {
this.splits.execute(new SplitRequest(r, midKey, this.server, user));
splitCounter += 1;
if (LOG.isDebugEnabled()) {
LOG.debug("Splitting " + r + ", " + this);
}
Expand Down Expand Up @@ -503,12 +509,13 @@ public int getSplitQueueSize() {
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()
&& ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the key change, every other part of this PR is just unit testing

Copy link

Copilot AI Jun 1, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[nitpick] Since this condition ensures that only primary replicas are allowed to be split, consider adding a brief code comment or updating the documentation to clarify this behavior for future maintainers.

Copilot uses AI. Check for mistakes.
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

}

/** Returns the regionSplitLimit */
Expand Down Expand Up @@ -807,6 +814,11 @@ protected int getSplitThreadNum() {
return this.splits.getCorePoolSize();
}

/** Exposed for unit testing */
int getSplitCounter() {
return splitCounter;
}

/**
* {@inheritDoc}
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,8 @@
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;
Expand All @@ -64,8 +63,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();
Expand All @@ -92,12 +91,7 @@ private static void setupConf(Configuration conf) {
}

@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) {
Expand Down Expand Up @@ -173,4 +167,22 @@ public void testFlushWithTableCompactionDisabled() throws Exception {
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().getSplitCounter());
}
}