HBASE-14227 Fold special cased MOB APIs into existing APIs

Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
chenheng 2015-09-25 12:45:50 +08:00 committed by Andrew Purtell
parent daccb1c0c2
commit 02699fe967
9 changed files with 246 additions and 310 deletions

View File

@ -1543,56 +1543,61 @@ public interface Admin extends Abortable, Closeable {
public int getMasterInfoPort() throws IOException;
/**
* Compact the mob files in all mob-enabled column families. Asynchronous operation.
* Compact a table. Asynchronous operation.
*
* @param tableName table to compact
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType}
* @throws IOException
* @throws InterruptedException
*/
void compactMobs(final TableName tableName) throws IOException,
InterruptedException;
void compact(final TableName tableName, CompactType compactType)
throws IOException, InterruptedException;
/**
* Compact the mob files in a mob-enabled column family. Asynchronous operation.
* Compact a column family within a table. Asynchronous operation.
*
* @param tableName table to compact
* @param columnFamily column family within a table
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType}
* @throws IOException if not a mob column family or if a remote or network exception occurs
* @throws InterruptedException
*/
void compactMob(final TableName tableName, final byte[] columnFamily) throws IOException,
InterruptedException;
void compact(final TableName tableName, final byte[] columnFamily, CompactType compactType)
throws IOException, InterruptedException;
/**
* Major compact the mob files in all mob-enabled column family. Asynchronous operation.
* Major compact a table. Asynchronous operation.
*
* @param tableName table to compact
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType}
* @throws IOException
* @throws InterruptedException
*/
void majorCompactMobs(final TableName tableName) throws IOException,
InterruptedException;
void majorCompact(final TableName tableName, CompactType compactType)
throws IOException, InterruptedException;
/**
* Major compact the mob files in a mob-enabled column family. Asynchronous operation.
* Major compact a column family within a table. Asynchronous operation.
*
* @param tableName table to compact
* @param columnFamily column family within a table
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType}
* @throws IOException if not a mob column family or if a remote or network exception occurs
* @throws InterruptedException
*/
void majorCompactMob(final TableName tableName, final byte[] columnFamily) throws IOException,
InterruptedException;
void majorCompact(final TableName tableName, final byte[] columnFamily, CompactType compactType)
throws IOException, InterruptedException;
/**
* Get the current compaction state of a table. It could be in a compaction, or none.
*
* @param tableName table to examine
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType}
* @return the current compaction state
* @throws IOException if a remote or network exception occurs
*/
AdminProtos.GetRegionInfoResponse.CompactionState getMobCompactionState(final TableName tableName)
throws IOException;
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(final TableName tableName,
CompactType compactType) throws IOException;
/**
* Return the set of supported security capabilities.
@ -1600,4 +1605,20 @@ public interface Admin extends Abortable, Closeable {
* @throws UnsupportedOperationException
*/
List<SecurityCapability> getSecurityCapabilities() throws IOException;
/**
* Currently, there are only two compact types:
* {@code NORMAL} means do store files compaction;
* {@code MOB} means do mob files compaction.
* */
@InterfaceAudience.Public
@InterfaceStability.Unstable
public enum CompactType {
NORMAL (0),
MOB (1);
CompactType(int value) {}
}
}

View File

@ -1846,7 +1846,7 @@ public class HBaseAdmin implements Admin {
@Override
public void compact(final TableName tableName)
throws IOException {
compact(tableName, null, false);
compact(tableName, null, false, CompactType.NORMAL);
}
/**
@ -1878,7 +1878,7 @@ public class HBaseAdmin implements Admin {
try {
compactRegion(tableNameOrRegionName, null, false);
} catch (IllegalArgumentException e) {
compact(TableName.valueOf(tableNameOrRegionName), null, false);
compact(TableName.valueOf(tableNameOrRegionName), null, false, CompactType.NORMAL);
}
}
@ -1888,7 +1888,7 @@ public class HBaseAdmin implements Admin {
@Override
public void compact(final TableName tableName, final byte[] columnFamily)
throws IOException {
compact(tableName, columnFamily, false);
compact(tableName, columnFamily, false, CompactType.NORMAL);
}
/**
@ -1921,7 +1921,7 @@ public class HBaseAdmin implements Admin {
compactRegion(tableNameOrRegionName, columnFamily, false);
} catch (IllegalArgumentException e) {
// Bad region, try table
compact(TableName.valueOf(tableNameOrRegionName), columnFamily, false);
compact(TableName.valueOf(tableNameOrRegionName), columnFamily, false, CompactType.NORMAL);
}
}
@ -1942,7 +1942,7 @@ public class HBaseAdmin implements Admin {
@Override
public void majorCompact(final TableName tableName)
throws IOException {
compact(tableName, null, true);
compact(tableName, null, true, CompactType.NORMAL);
}
/**
@ -1975,7 +1975,7 @@ public class HBaseAdmin implements Admin {
compactRegion(tableNameOrRegionName, null, true);
} catch (IllegalArgumentException e) {
// Invalid region, try table
compact(TableName.valueOf(tableNameOrRegionName), null, true);
compact(TableName.valueOf(tableNameOrRegionName), null, true, CompactType.NORMAL);
}
}
@ -1985,7 +1985,7 @@ public class HBaseAdmin implements Admin {
@Override
public void majorCompact(final TableName tableName, final byte[] columnFamily)
throws IOException {
compact(tableName, columnFamily, true);
compact(tableName, columnFamily, true, CompactType.NORMAL);
}
/**
@ -2018,7 +2018,7 @@ public class HBaseAdmin implements Admin {
compactRegion(tableNameOrRegionName, columnFamily, true);
} catch (IllegalArgumentException e) {
// Invalid region, try table
compact(TableName.valueOf(tableNameOrRegionName), columnFamily, true);
compact(TableName.valueOf(tableNameOrRegionName), columnFamily, true, CompactType.NORMAL);
}
}
@ -2032,36 +2032,44 @@ public class HBaseAdmin implements Admin {
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
private void compact(final TableName tableName, final byte[] columnFamily,final boolean major)
throws IOException {
ZooKeeperWatcher zookeeper = null;
try {
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<HRegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
private void compact(final TableName tableName, final byte[] columnFamily,final boolean major,
CompactType compactType) throws IOException {
switch (compactType) {
case MOB:
ServerName master = getMasterAddress();
compact(master, getMobRegionInfo(tableName), major, columnFamily);
break;
case NORMAL:
ZooKeeperWatcher zookeeper = null;
try {
compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
pair.getFirst() + ": " +
StringUtils.stringifyException(e));
checkTableExists(tableName);
zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
List<Pair<HRegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
pair.getFirst() + ": " +
StringUtils.stringifyException(e));
}
}
}
} finally {
if (zookeeper != null) {
zookeeper.close();
}
}
}
} finally {
if (zookeeper != null) {
zookeeper.close();
}
break;
}
}
@ -3100,68 +3108,7 @@ public class HBaseAdmin implements Admin {
@Override
public CompactionState getCompactionState(final TableName tableName)
throws IOException {
CompactionState state = CompactionState.NONE;
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
try {
checkTableExists(tableName);
List<Pair<HRegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
for (Pair<HRegionInfo, ServerName> pair: pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
ServerName sn = pair.getSecond();
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
pair.getFirst().getRegionName(), true);
GetRegionInfoResponse response = admin.getRegionInfo(null, request);
switch (response.getCompactionState()) {
case MAJOR_AND_MINOR:
return CompactionState.MAJOR_AND_MINOR;
case MAJOR:
if (state == CompactionState.MINOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = CompactionState.MAJOR;
break;
case MINOR:
if (state == CompactionState.MAJOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = CompactionState.MINOR;
break;
case NONE:
default: // nothing, continue
}
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " +
pair.getFirst() + ": " +
StringUtils.stringifyException(e));
}
} catch (RemoteException e) {
if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
+ StringUtils.stringifyException(e));
}
} else {
throw e;
}
}
}
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
} finally {
zookeeper.close();
}
return state;
return getCompactionState(tableName, CompactType.NORMAL);
}
/**
@ -4222,6 +4169,18 @@ public class HBaseAdmin implements Admin {
}
}
private ServerName getMasterAddress() throws IOException {
// TODO: Fix! Reaching into internal implementation!!!!
ConnectionImplementation connection =
(ConnectionImplementation)this.connection;
ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
try {
return MasterAddressTracker.getMasterAddress(zkw);
} catch (KeeperException e) {
throw new IOException("Failed to get master server name from MasterAddressTracker", e);
}
}
@Override
public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
return executeCallable(new MasterCallable<Long>(getConnection()) {
@ -4255,102 +4214,123 @@ public class HBaseAdmin implements Admin {
* {@inheritDoc}
*/
@Override
public void compactMob(final TableName tableName, final byte[] columnFamily)
public void compact(final TableName tableName, final byte[] columnFamily, CompactType compactType)
throws IOException, InterruptedException {
checkTableNameNotNull(tableName);
checkFamilyNameNotNull(columnFamily);
validateMobColumnFamily(tableName, columnFamily);
compactMob(tableName, columnFamily, false);
compact(tableName, columnFamily, false, compactType);
}
/**
* {@inheritDoc}
*/
@Override
public void compactMobs(final TableName tableName) throws IOException, InterruptedException {
checkTableNameNotNull(tableName);
compactMob(tableName, null, false);
}
/**
* {@inheritDoc}
*/
@Override
public void majorCompactMob(final TableName tableName, final byte[] columnFamily)
public void compact(final TableName tableName, CompactType compactType)
throws IOException, InterruptedException {
checkTableNameNotNull(tableName);
checkFamilyNameNotNull(columnFamily);
validateMobColumnFamily(tableName, columnFamily);
compactMob(tableName, columnFamily, true);
compact(tableName, null, false, compactType);
}
/**
* {@inheritDoc}
*/
@Override
public void majorCompactMobs(final TableName tableName) throws IOException, InterruptedException {
checkTableNameNotNull(tableName);
compactMob(tableName, null, true);
public void majorCompact(final TableName tableName, final byte[] columnFamily,
CompactType compactType) throws IOException, InterruptedException {
compact(tableName, columnFamily, true, compactType);
}
/**
* {@inheritDoc}
*/
@Override
public CompactionState getMobCompactionState(TableName tableName) throws IOException {
checkTableNameNotNull(tableName);
try {
ServerName master = getClusterStatus().getMaster();
HRegionInfo info = new HRegionInfo(tableName, Bytes.toBytes(".mob"),
HConstants.EMPTY_END_ROW, false, 0);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
info.getRegionName(), true);
GetRegionInfoResponse response = this.connection.getAdmin(master)
.getRegionInfo(null, request);
return response.getCompactionState();
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
public void majorCompact(final TableName tableName, CompactType compactType)
throws IOException, InterruptedException {
compact(tableName, null, true, compactType);
}
/**
* Compacts the mob files in a mob-enabled column family. Asynchronous operation.
* @param tableName The table to compact.
* @param columnFamily The column family to compact. If it is null, all the mob-enabled
* column families in this table will be compacted.
* @param major Whether to select all the mob files in the compaction.
* @throws IOException
* @throws InterruptedException
* {@inheritDoc}
*/
private void compactMob(final TableName tableName, final byte[] columnFamily, boolean major)
throws IOException, InterruptedException {
// get the mob region info, this is a dummy region.
HRegionInfo info = new HRegionInfo(tableName, Bytes.toBytes(".mob"), HConstants.EMPTY_END_ROW,
false, 0);
ServerName master = getClusterStatus().getMaster();
compact(master, info, major, columnFamily);
}
private void checkTableNameNotNull(TableName tableName) {
if (tableName == null) {
throw new IllegalArgumentException("TableName cannot be null");
}
}
private void checkFamilyNameNotNull(byte[] columnFamily) {
if (columnFamily == null) {
throw new IllegalArgumentException("The column family name cannot be null");
}
}
private void validateMobColumnFamily(TableName tableName, byte[] columnFamily)
throws IOException {
HTableDescriptor htd = getTableDescriptor(tableName);
HColumnDescriptor family = htd.getFamily(columnFamily);
if (family == null || !family.isMobEnabled()) {
throw new IllegalArgumentException("Column family " + Bytes.toString(columnFamily)
+ " is not a mob column family");
@Override
public CompactionState getCompactionState(TableName tableName,
CompactType compactType) throws IOException {
CompactionState state = CompactionState.NONE;
checkTableExists(tableName);
switch (compactType) {
case MOB:
try {
ServerName master = getMasterAddress();
HRegionInfo info = getMobRegionInfo(tableName);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
info.getRegionName(), true);
GetRegionInfoResponse response = this.connection.getAdmin(master)
.getRegionInfo(null, request);
state = response.getCompactionState();
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
break;
case NORMAL:
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
new ThrowableAbortable());
try {
List<Pair<HRegionInfo, ServerName>> pairs;
if (TableName.META_TABLE_NAME.equals(tableName)) {
pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper);
} else {
pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName);
}
for (Pair<HRegionInfo, ServerName> pair : pairs) {
if (pair.getFirst().isOffline()) continue;
if (pair.getSecond() == null) continue;
try {
ServerName sn = pair.getSecond();
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
pair.getFirst().getRegionName(), true);
GetRegionInfoResponse response = admin.getRegionInfo(null, request);
switch (response.getCompactionState()) {
case MAJOR_AND_MINOR:
return CompactionState.MAJOR_AND_MINOR;
case MAJOR:
if (state == CompactionState.MINOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = CompactionState.MAJOR;
break;
case MINOR:
if (state == CompactionState.MAJOR) {
return CompactionState.MAJOR_AND_MINOR;
}
state = CompactionState.MINOR;
break;
case NONE:
default: // nothing, continue
}
} catch (NotServingRegionException e) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " +
pair.getFirst() + ": " +
StringUtils.stringifyException(e));
}
} catch (RemoteException e) {
if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
+ StringUtils.stringifyException(e));
}
} else {
throw e;
}
}
}
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
} finally {
zookeeper.close();
}
break;
}
return state;
}
/**
@ -4800,4 +4780,10 @@ public class HBaseAdmin implements Admin {
throw e;
}
}
private HRegionInfo getMobRegionInfo(TableName tableName) {
return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
HConstants.EMPTY_END_ROW, false, 0);
}
}

View File

@ -51,9 +51,9 @@ public class CompactMobAction extends Action {
LOG.info("Performing action: Compact mob of table " + tableName + ", major=" + major);
try {
if (major) {
admin.majorCompactMobs(tableName);
admin.majorCompact(tableName, Admin.CompactType.MOB);
} else {
admin.compactMobs(tableName);
admin.compact(tableName, Admin.CompactType.MOB);
}
} catch (Exception ex) {
LOG.warn("Mob Compaction failed, might be caused by other chaos: " + ex.getMessage());

View File

@ -570,7 +570,7 @@ public class TestMobCompactor {
int largeFilesCount = countLargeFiles(5000, family1);
// do the mob compaction
admin.compactMob(tableName, hcd1.getName());
admin.compact(tableName, hcd1.getName(), Admin.CompactType.MOB);
waitUntilMobCompactionFinished(tableName);
assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
@ -618,7 +618,7 @@ public class TestMobCompactor {
countFiles(tableName, false, family2));
// do the major mob compaction, it will force all files to compaction
admin.majorCompactMob(tableName, hcd1.getName());
admin.majorCompact(tableName, hcd1.getName(), Admin.CompactType.MOB);
waitUntilMobCompactionFinished(tableName);
assertEquals("After compaction: mob rows count", regionNum*(rowNumPerRegion-delRowNum),
@ -657,7 +657,7 @@ public class TestMobCompactor {
Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1));
assertEquals("Before compaction: mob value of k0", newValue0,
Bytes.toString(CellUtil.cloneValue(cell)));
admin.majorCompactMob(tableName, hcd1.getName());
admin.majorCompact(tableName, hcd1.getName(), Admin.CompactType.MOB);
waitUntilMobCompactionFinished(tableName);
// read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the
// scanner. The cell that has "new" value is still visible.
@ -705,7 +705,7 @@ public class TestMobCompactor {
loadData(admin, bufMut, tableName, new Put[] { put1 }); // now two mob files
admin.majorCompact(tableName);
waitUntilCompactionFinished(tableName);
admin.majorCompactMob(tableName, hcd1.getName());
admin.majorCompact(tableName, hcd1.getName(), Admin.CompactType.MOB);
waitUntilMobCompactionFinished(tableName);
// read the latest cell of key1.
Get get = new Get(key1);
@ -731,12 +731,12 @@ public class TestMobCompactor {
private void waitUntilMobCompactionFinished(TableName tableName) throws IOException,
InterruptedException {
long finished = EnvironmentEdgeManager.currentTime() + 60000;
CompactionState state = admin.getMobCompactionState(tableName);
CompactionState state = admin.getCompactionState(tableName, Admin.CompactType.MOB);
while (EnvironmentEdgeManager.currentTime() < finished) {
if (state == CompactionState.NONE) {
break;
}
state = admin.getMobCompactionState(tableName);
state = admin.getCompactionState(tableName, Admin.CompactType.MOB);
Thread.sleep(10);
}
assertEquals(CompactionState.NONE, state);

View File

@ -56,12 +56,25 @@ module Hbase
#----------------------------------------------------------------------------------------------
# Requests a table or region or column family compaction
def compact(table_or_region_name, family = nil)
if family == nil
@admin.compact(table_or_region_name)
def compact(table_or_region_name, family = nil, type = "NORMAL")
if type == "NORMAL"
if family == nil
@admin.compact(table_or_region_name)
else
# We are compacting a column family within a region.
@admin.compact(table_or_region_name, family)
end
elsif type == "MOB"
if family == nil
@admin.compact(org.apache.hadoop.hbase.TableName.valueOf(table_or_region_name),
org.apache.hadoop.hbase.client.Admin::CompactType::MOB)
else
# We are compacting a mob column family within a table.
@admin.compact(org.apache.hadoop.hbase.TableName.valueOf(table_or_region_name), family.to_java_bytes,
org.apache.hadoop.hbase.client.Admin::CompactType::MOB)
end
else
# We are compacting a column family within a region.
@admin.compact(table_or_region_name, family)
raise ArgumentError, "only NORMAL or MOB accepted for type!"
end
end
@ -72,12 +85,25 @@ module Hbase
#----------------------------------------------------------------------------------------------
# Requests a table or region or column family major compaction
def major_compact(table_or_region_name, family = nil)
if family == nil
@admin.majorCompact(table_or_region_name)
def major_compact(table_or_region_name, family = nil, type = "NORMAL")
if type == "NORMAL"
if family == nil
@admin.majorCompact(table_or_region_name)
else
# We are major compacting a column family within a region or table.
@admin.majorCompact(table_or_region_name, family)
end
elsif type == "MOB"
if family == nil
@admin.majorCompact(org.apache.hadoop.hbase.TableName.valueOf(table_or_region_name),
org.apache.hadoop.hbase.client.Admin::CompactType::MOB)
else
# We are major compacting a mob column family within a table.
@admin.majorCompact(org.apache.hadoop.hbase.TableName.valueOf(table_or_region_name),
family.to_java_bytes, org.apache.hadoop.hbase.client.Admin::CompactType::MOB)
end
else
# We are major compacting a column family within a region or table.
@admin.majorCompact(table_or_region_name, family)
raise ArgumentError, "only NORMAL or MOB accepted for type!"
end
end
@ -993,28 +1019,6 @@ module Hbase
@admin.deleteNamespace(namespace_name)
end
#----------------------------------------------------------------------------------------------
# Requests a mob file compaction
def compact_mob(table_name, family = nil)
if family == nil
@admin.compactMobs(org.apache.hadoop.hbase.TableName.valueOf(table_name))
else
# We are compacting a mob column family within a table.
@admin.compactMob(org.apache.hadoop.hbase.TableName.valueOf(table_name), family.to_java_bytes)
end
end
#----------------------------------------------------------------------------------------------
# Requests a mob file major compaction
def major_compact_mob(table_name, family = nil)
if family == nil
@admin.majorCompactMobs(org.apache.hadoop.hbase.TableName.valueOf(table_name))
else
# We are major compacting a mob column family within a table.
@admin.majorCompactMob(org.apache.hadoop.hbase.TableName.valueOf(table_name), family.to_java_bytes)
end
end
#----------------------------------------------------------------------------------------------
# Get security capabilities
def get_security_capabilities

View File

@ -25,6 +25,7 @@ module Shell
Compact all regions in passed table or pass a region row
to compact an individual region. You can also compact a single column
family within a region.
You can also set compact type, "NORMAL" or "MOB", and default is "NORMAL"
Examples:
Compact all regions in a table:
hbase> compact 'ns1:t1'
@ -35,12 +36,16 @@ module Shell
hbase> compact 'r1', 'c1'
Compact a column family within a table:
hbase> compact 't1', 'c1'
Compact table with type "MOB"
hbase> compact 't1', nil, 'MOB'
Compact a column family using "MOB" type within a table
hbase> compact 't1', 'c1', 'MOB'
EOF
end
def command(table_or_region_name, family = nil)
def command(table_or_region_name, family = nil, type = "NORMAL")
format_simple_command do
admin.compact(table_or_region_name, family)
admin.compact(table_or_region_name, family, type)
end
end
end

View File

@ -1,42 +0,0 @@
#
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
module Shell
module Commands
class CompactMob < Command
def help
return <<-EOF
Run compaction on a mob enabled column family
or all mob enabled column families within a table
Examples:
Compact a column family within a table:
hbase> compact_mob 't1', 'c1'
Compact all mob enabled column families
hbase> compact_mob 't1'
EOF
end
def command(table_name, family = nil)
format_simple_command do
admin.compact_mob(table_name, family)
end
end
end
end
end

View File

@ -36,12 +36,16 @@ module Shell
hbase> major_compact 'r1', 'c1'
Compact a single column family within a table:
hbase> major_compact 't1', 'c1'
Compact table with type "MOB"
hbase> major_compact 't1', nil, 'MOB'
Compact a column family using "MOB" type within a table
hbase> major_compact 't1', 'c1', 'MOB'
EOF
end
def command(table_or_region_name, family = nil)
def command(table_or_region_name, family = nil, type = "NORMAL")
format_simple_command do
admin.major_compact(table_or_region_name, family)
admin.major_compact(table_or_region_name, family, type)
end
end
end

View File

@ -1,42 +0,0 @@
#
#
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
module Shell
module Commands
class MajorCompactMob < Command
def help
return <<-EOF
Run major compaction on a mob enabled column family
or all mob enabled column families within a table
Examples:
Compact a column family within a table:
hbase> major_compact_mob 't1', 'c1'
Compact all mob enabled column families within a table
hbase> major_compact_mob 't1'
EOF
end
def command(table_name, family = nil)
format_simple_command do
admin.major_compact_mob(table_name, family)
end
end
end
end
end