HBASE-7875 introduce a compaction switch in HBase Shell (Liang Xie)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1505060 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
sershe 2013-07-19 22:40:31 +00:00
parent 2fb6b8b7f3
commit 245cd38983
3 changed files with 40 additions and 2 deletions

View File

@ -124,6 +124,16 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
private static final ImmutableBytesWritable READONLY_KEY =
new ImmutableBytesWritable(Bytes.toBytes(READONLY));
/**
* <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
* attribute which denotes if the table is compaction enabled
*
* @see #isCompactionEnabled()
*/
public static final String COMPACTION_ENABLED = "COMPACTION_ENABLED";
private static final ImmutableBytesWritable COMPACTION_ENABLED_KEY =
new ImmutableBytesWritable(Bytes.toBytes(COMPACTION_ENABLED));
/**
* <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
* attribute which represents the maximum size of the memstore after which
@ -195,6 +205,11 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
*/
public static final boolean DEFAULT_READONLY = false;
/**
* Constant that denotes whether the table is compaction enabled by default
*/
public static final boolean DEFAULT_COMPACTION_ENABLED = true;
/**
* Constant that denotes the maximum default size of the memstore after which
* the contents are flushed to the store files
@ -611,6 +626,25 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
setValue(READONLY_KEY, readOnly? TRUE: FALSE);
}
/**
* Check if the compaction enable flag of the table is true. If flag is
* false then no minor/major compactions will be done in real.
*
* @return true if table compaction enabled
*/
public boolean isCompactionEnabled() {
return isSomething(COMPACTION_ENABLED_KEY, DEFAULT_COMPACTION_ENABLED);
}
/**
* Setting the table compaction enable flag.
*
* @param isEnable True if enable compaction.
*/
public void setCompactionEnabled(final boolean isEnable) {
setValue(COMPACTION_ENABLED_KEY, isEnable ? TRUE : FALSE);
}
/**
* Check if deferred log edits are enabled on the table.
*

View File

@ -297,7 +297,8 @@ public class CompactSplitThread implements CompactionRequestor {
private synchronized CompactionRequest requestCompactionInternal(final HRegion r, final Store s,
final String why, int priority, CompactionRequest request, boolean selectNow)
throws IOException {
if (this.server.isStopped()) {
if (this.server.isStopped()
|| (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
return null;
}
@ -418,7 +419,8 @@ public class CompactSplitThread implements CompactionRequestor {
@Override
public void run() {
Preconditions.checkNotNull(server);
if (server.isStopped()) {
if (server.isStopped()
|| (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
return;
}
// Common case - system compaction without a file selection. Select now.

View File

@ -259,6 +259,7 @@ module Hbase
htd.setOwnerString(arg.delete(OWNER)) if arg[OWNER]
htd.setMaxFileSize(JLong.valueOf(arg.delete(MAX_FILESIZE))) if arg[MAX_FILESIZE]
htd.setReadOnly(JBoolean.valueOf(arg.delete(READONLY))) if arg[READONLY]
htd.setCompactionEnabled(JBoolean.valueOf(arg[COMPACTION_ENABLED])) if arg[COMPACTION_ENABLED]
htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
htd.setDeferredLogFlush(JBoolean.valueOf(arg.delete(DEFERRED_LOG_FLUSH))) if arg[DEFERRED_LOG_FLUSH]
htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
@ -468,6 +469,7 @@ module Hbase
htd.setOwnerString(arg.delete(OWNER)) if arg[OWNER]
htd.setMaxFileSize(JLong.valueOf(arg.delete(MAX_FILESIZE))) if arg[MAX_FILESIZE]
htd.setReadOnly(JBoolean.valueOf(arg.delete(READONLY))) if arg[READONLY]
htd.setCompactionEnabled(JBoolean.valueOf(arg[COMPACTION_ENABLED])) if arg[COMPACTION_ENABLED]
htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
htd.setDeferredLogFlush(JBoolean.valueOf(arg.delete(DEFERRED_LOG_FLUSH))) if arg[DEFERRED_LOG_FLUSH]
htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]