HBASE-4414 Region splits by size not being triggered
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1171366 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
848748b1ae
commit
eca8e0a8f6
|
@ -271,6 +271,7 @@ Release 0.91.0 - Unreleased
|
|||
HBASE-4390 [replication] ReplicationSource's UncaughtExceptionHandler
|
||||
shouldn't join
|
||||
HBASE-4395 EnableTableHandler races with itself
|
||||
HBASE-4414 Region splits by size not being triggered
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
|
||||
|
|
|
@ -185,6 +185,9 @@ public class CompactionRequest implements Comparable<CompactionRequest>,
|
|||
if (s.getCompactPriority() <= 0) {
|
||||
server.compactSplitThread
|
||||
.requestCompaction(r, s, "Recursive enqueue");
|
||||
} else {
|
||||
// see if the compaction has caused us to exceed max region size
|
||||
server.compactSplitThread.requestSplit(r);
|
||||
}
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
|
|
Loading…
Reference in New Issue