HBASE-17280 Add mechanism to control hbase cleaner behavior

Signed-off-by: anoopsamjohn <anoopsamjohn@gmail.com>
Signed-off-by: tedyu <yuzhihong@gmail.com>
This commit is contained in:
Ajay Jadhav 2017-02-01 11:33:01 -08:00 committed by Sean Busbey
parent 2c799fb70a
commit f8b1f57b05
16 changed files with 3963 additions and 568 deletions

View File

@ -896,6 +896,30 @@ public interface Admin extends Abortable, Closeable {
*/
boolean isCatalogJanitorEnabled() throws IOException;
/**
* Enable/Disable the cleaner chore
*
* @param on if true enables the cleaner chore
* @return the previous state
* @throws IOException
*/
public boolean setCleanerChoreRunning(final boolean on) throws IOException;
/**
* Ask for cleaner chore to run
*
* @return True if cleaner chore ran, false otherwise
* @throws IOException
*/
public boolean runCleanerChore() throws IOException;
/**
* Query on the cleaner chore state (Enabled/Disabled?)
*
* @throws IOException
*/
public boolean isCleanerChoreEnabled() throws IOException;
/**
* Merge two regions. Asynchronous operation.
*

View File

@ -80,12 +80,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientServ
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
@ -1443,6 +1443,27 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return stub.isCatalogJanitorEnabled(controller, request);
}
@Override
public MasterProtos.RunCleanerChoreResponse runCleanerChore(RpcController controller,
MasterProtos.RunCleanerChoreRequest request)
throws ServiceException {
return stub.runCleanerChore(controller, request);
}
@Override
public MasterProtos.SetCleanerChoreRunningResponse setCleanerChoreRunning(
RpcController controller, MasterProtos.SetCleanerChoreRunningRequest request)
throws ServiceException {
return stub.setCleanerChoreRunning(controller, request);
}
@Override
public MasterProtos.IsCleanerChoreEnabledResponse isCleanerChoreEnabled(
RpcController controller, MasterProtos.IsCleanerChoreEnabledRequest request)
throws ServiceException {
return stub.isCleanerChoreEnabled(controller, request);
}
@Override
public ClientProtos.CoprocessorServiceResponse execMasterService(
RpcController controller, ClientProtos.CoprocessorServiceRequest request)

View File

@ -58,8 +58,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
@ -1459,6 +1459,40 @@ public class HBaseAdmin implements Admin {
});
}
@Override
public boolean setCleanerChoreRunning(final boolean on) throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
@Override public Boolean rpcCall() throws Exception {
return master.setCleanerChoreRunning(getRpcController(), RequestConverter
.buildSetCleanerChoreRunningRequest(
on)).getPrevValue();
}
});
}
@Override
public boolean runCleanerChore() throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
@Override public Boolean rpcCall() throws Exception {
return master
.runCleanerChore(getRpcController(), RequestConverter
.buildRunCleanerChoreRequest())
.getCleanerChoreRan();
}
});
}
@Override
public boolean isCleanerChoreEnabled() throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection(), getRpcControllerFactory()) {
@Override public Boolean rpcCall() throws Exception {
return master.isCleanerChoreEnabled(getRpcController(),
RequestConverter.buildIsCleanerChoreEnabledRequest())
.getValue();
}
});
}
private boolean isEncodedRegionName(byte[] regionName) throws IOException {
try {
HRegionInfo.parseRegionName(regionName);

View File

@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColu
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
@ -93,6 +94,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNa
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
@ -103,6 +105,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegion
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
@ -1451,6 +1454,42 @@ public final class RequestConverter {
return IS_CATALOG_JANITOR_ENABLED_REQUEST;
}
/**
* @see {@link #buildCleanerChoreRequest}
*/
private static final RunCleanerChoreRequest CLEANER_CHORE_REQUEST =
RunCleanerChoreRequest.newBuilder().build();
/**
* Creates a request for running cleaner chore
* @return A {@link RunCleanerChoreRequest}
*/
public static RunCleanerChoreRequest buildRunCleanerChoreRequest() {
return CLEANER_CHORE_REQUEST;
}
/**
* Creates a request for enabling/disabling the cleaner chore
* @return A {@link SetCleanerChoreRunningRequest}
*/
public static SetCleanerChoreRunningRequest buildSetCleanerChoreRunningRequest(boolean on) {
return SetCleanerChoreRunningRequest.newBuilder().setOn(on).build();
}
/**
* @see {@link #buildIsCleanerChoreEnabledRequest()}
*/
private static final IsCleanerChoreEnabledRequest IS_CLEANER_CHORE_ENABLED_REQUEST =
IsCleanerChoreEnabledRequest.newBuilder().build();
/**
* Creates a request for querying the master whether the cleaner chore is enabled
* @return A {@link IsCleanerChoreEnabledRequest}
*/
public static IsCleanerChoreEnabledRequest buildIsCleanerChoreEnabledRequest() {
return IS_CLEANER_CHORE_ENABLED_REQUEST;
}
/**
* Creates a request for querying the master the last flushed sequence Id for a region
* @param regionName

View File

@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameInt64Pa
import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMetrics;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
@ -314,6 +315,14 @@ public final class ResponseConverter {
return EnableCatalogJanitorResponse.newBuilder().setPrevValue(prevValue).build();
}
/**
* Creates a response for the cleaner chore request
* @return A RunCleanerChoreResponse
*/
public static RunCleanerChoreResponse buildRunCleanerChoreResponse(boolean ran) {
return RunCleanerChoreResponse.newBuilder().setCleanerChoreRan(ran).build();
}
// End utilities for Admin
/**

View File

@ -356,6 +356,28 @@ message IsCatalogJanitorEnabledResponse {
required bool value = 1;
}
message RunCleanerChoreRequest {
}
message RunCleanerChoreResponse {
required bool cleaner_chore_ran = 1;
}
message SetCleanerChoreRunningRequest {
required bool on = 1;
}
message SetCleanerChoreRunningResponse {
optional bool prev_value = 1;
}
message IsCleanerChoreEnabledRequest {
}
message IsCleanerChoreEnabledResponse {
required bool value = 1;
}
message SnapshotRequest {
required SnapshotDescription snapshot = 1;
}
@ -742,6 +764,22 @@ service MasterService {
rpc IsCatalogJanitorEnabled(IsCatalogJanitorEnabledRequest)
returns(IsCatalogJanitorEnabledResponse);
/** Get a run of the CleanerChore */
rpc RunCleanerChore(RunCleanerChoreRequest)
returns(RunCleanerChoreResponse);
/**
* Enable the CleanerChore on or off.
*/
rpc SetCleanerChoreRunning(SetCleanerChoreRunningRequest)
returns(SetCleanerChoreRunningResponse);
/**
* Query whether the CleanerChore is enabled.
*/
rpc IsCleanerChoreEnabled(IsCleanerChoreEnabledRequest)
returns(IsCleanerChoreEnabledResponse);
/**
* Call a master coprocessor endpoint
*/

View File

@ -97,9 +97,9 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.cleaner.ReplicationMetaCleaner;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleanerChore;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
@ -113,9 +113,9 @@ import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.master.procedure.MergeTableRegionsProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
@ -954,6 +954,20 @@ public class HMaster extends HRegionServer implements MasterServices {
catalogJanitorChore.getEnabled() : false;
}
boolean isCleanerChoreEnabled() {
boolean hfileCleanerFlag = true, logCleanerFlag = true;
if (hfileCleaner != null) {
hfileCleanerFlag = hfileCleaner.getEnabled();
}
if (logCleaner != null) {
logCleanerFlag = logCleaner.getEnabled();
}
return (hfileCleanerFlag && logCleanerFlag);
}
@Override
public TableDescriptors getTableDescriptors() {
return this.tableDescriptors;
@ -2660,6 +2674,10 @@ public class HMaster extends HRegionServer implements MasterServices {
return this.hfileCleaner;
}
public LogCleaner getLogCleaner() {
return this.logCleaner;
}
/**
* @return the underlying snapshot manager
*/

View File

@ -520,6 +520,22 @@ public class MasterRpcServices extends RSRpcServices
master.catalogJanitorChore.setEnabled(req.getEnable())).build();
}
@Override
public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController c,
SetCleanerChoreRunningRequest req)
throws ServiceException {
try {
master.checkInitialized();
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
boolean prevValue =
master.getLogCleaner().getEnabled() && master.getHFileCleaner().getEnabled();
master.getLogCleaner().setEnabled(req.getOn());
master.getHFileCleaner().setEnabled(req.getOn());
return SetCleanerChoreRunningResponse.newBuilder().setPrevValue(prevValue).build();
}
@Override
public EnableTableResponse enableTable(RpcController controller,
EnableTableRequest request) throws ServiceException {
@ -872,6 +888,14 @@ public class MasterRpcServices extends RSRpcServices
master.isCatalogJanitorEnabled()).build();
}
@Override
public IsCleanerChoreEnabledResponse isCleanerChoreEnabled(RpcController c,
IsCleanerChoreEnabledRequest req)
throws ServiceException {
return IsCleanerChoreEnabledResponse.newBuilder().setValue(master.isCleanerChoreEnabled())
.build();
}
@Override
public IsMasterRunningResponse isMasterRunning(RpcController c,
IsMasterRunningRequest req) throws ServiceException {
@ -1202,6 +1226,18 @@ public class MasterRpcServices extends RSRpcServices
}
}
@Override
public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
throws ServiceException {
try {
master.checkInitialized();
Boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
return ResponseConverter.buildRunCleanerChoreResponse(result);
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
}
@Override
public SetBalancerRunningResponse setBalancerRunning(RpcController c,
SetBalancerRunningRequest req) throws ServiceException {

View File

@ -36,6 +36,7 @@ import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Abstract Cleaner that uses a chain of delegates to clean a directory of files
@ -50,6 +51,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
private final Configuration conf;
protected List<T> cleanersChain;
protected Map<String, Object> params;
private AtomicBoolean enabled = new AtomicBoolean(true);
public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf,
FileSystem fs, Path oldFileDir, String confKey) {
@ -128,6 +130,14 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
@Override
protected void chore() {
if (getEnabled()) {
runCleaner();
} else {
LOG.debug("Cleaner chore disabled! Not cleaning.");
}
}
public Boolean runCleaner() {
try {
FileStatus[] files = FSUtils.listStatus(this.fs, this.oldFileDir);
checkAndDeleteEntries(files);
@ -135,7 +145,9 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
e = e instanceof RemoteException ?
((RemoteException)e).unwrapRemoteException() : e;
LOG.warn("Error while cleaning the logs", e);
return false;
}
return true;
}
/**
@ -291,4 +303,15 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
}
}
}
/**
* @param enabled
*/
public boolean setEnabled(final boolean enabled) {
return this.enabled.getAndSet(enabled);
}
public boolean getEnabled() {
return this.enabled.get();
}
}

View File

@ -289,6 +289,72 @@ public class TestCleanerChore {
Mockito.verify(spy, Mockito.times(1)).isFileDeletable(Mockito.any(FileStatus.class));
}
@Test
public void testDeleteFileWithCleanerEnabled() throws Exception {
Stoppable stop = new StoppableImplementation();
Configuration conf = UTIL.getConfiguration();
Path testDir = UTIL.getDataTestDir();
FileSystem fs = UTIL.getTestFileSystem();
String confKey = "hbase.test.cleaner.delegates";
conf.set(confKey, AlwaysDelete.class.getName());
AllValidPaths chore = new AllValidPaths("test-file-cleaner", stop, conf, fs, testDir, confKey);
// Enable cleaner
chore.setEnabled(true);
// create the directory layout in the directory to clean
Path parent = new Path(testDir, "parent");
Path child = new Path(parent, "child");
Path file = new Path(child, "someFile");
fs.mkdirs(child);
// touch a new file
fs.create(file).close();
assertTrue("Test file didn't get created.", fs.exists(file));
// run the chore
chore.chore();
// verify all the files got deleted
assertFalse("File didn't get deleted", fs.exists(file));
assertFalse("Empty directory didn't get deleted", fs.exists(child));
assertFalse("Empty directory didn't get deleted", fs.exists(parent));
}
@Test
public void testDeleteFileWithCleanerDisabled() throws Exception {
Stoppable stop = new StoppableImplementation();
Configuration conf = UTIL.getConfiguration();
Path testDir = UTIL.getDataTestDir();
FileSystem fs = UTIL.getTestFileSystem();
String confKey = "hbase.test.cleaner.delegates";
conf.set(confKey, AlwaysDelete.class.getName());
AllValidPaths chore = new AllValidPaths("test-file-cleaner", stop, conf, fs, testDir, confKey);
// Disable cleaner
chore.setEnabled(false);
// create the directory layout in the directory to clean
Path parent = new Path(testDir, "parent");
Path child = new Path(parent, "child");
Path file = new Path(child, "someFile");
fs.mkdirs(child);
// touch a new file
fs.create(file).close();
assertTrue("Test file didn't get created.", fs.exists(file));
// run the chore
chore.chore();
// verify all the files exist
assertTrue("File got deleted with cleaner disabled", fs.exists(file));
assertTrue("Directory got deleted", fs.exists(child));
assertTrue("Directory got deleted", fs.exists(parent));
}
private static class AllValidPaths extends CleanerChore<BaseHFileCleanerDelegate> {
public AllValidPaths(String name, Stoppable s, Configuration conf, FileSystem fs,

View File

@ -242,6 +242,26 @@ module Hbase
@admin.isCatalogJanitorEnabled()
end
#----------------------------------------------------------------------------------------------
# Request cleaner chore to run (for garbage collection of HFiles and WAL files)
def cleaner_chore_run()
@admin.runCleanerChore()
end
#----------------------------------------------------------------------------------------------
# Enable/disable the cleaner chore
# Returns previous cleaner switch setting.
def cleaner_chore_switch(enableDisable)
@admin.setCleanerChoreRunning(java.lang.Boolean::valueOf(enableDisable))
end
#----------------------------------------------------------------------------------------------
# Query on the cleaner chore state (enabled/disabled?)
# Returns cleaner state (true signifies enabled).
def cleaner_chore_enabled()
@admin.isCleanerChoreEnabled()
end
#----------------------------------------------------------------------------------------------
# Enables a table
def enable(table_name)

View File

@ -348,6 +348,9 @@ Shell.load_command_group(
catalogjanitor_run
catalogjanitor_switch
catalogjanitor_enabled
cleaner_chore_run
cleaner_chore_switch
cleaner_chore_enabled
compact_rs
compaction_state
trace

View File

@ -0,0 +1,36 @@
#
# 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 CleanerChoreEnabled < Command
def help
return <<-EOF
Query for the Cleaner chore state (enabled/disabled?).
Examples:
hbase> cleaner_chore_enabled
EOF
end
def command()
formatter.row([admin.cleaner_chore_enabled()? "true" : "false"])
end
end
end
end

View File

@ -0,0 +1,35 @@
#
# 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 CleanerChoreRun < Command
def help
return <<-EOF
Cleaner chore command for garbage collection of HFiles and WAL files.
hbase> cleaner_chore_run
EOF
end
def command()
admin.cleaner_chore_run()
end
end
end
end

View File

@ -0,0 +1,37 @@
#
# 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 CleanerChoreSwitch < Command
def help
return <<-EOF
Enable/Disable Cleaner chore. Returns previous Cleaner chore state.
Examples:
hbase> cleaner_chore_switch true
hbase> cleaner_chore_switch false
EOF
end
def command(enableDisable)
formatter.row([admin.cleaner_chore_switch(enableDisable)? "true" : "false"])
end
end
end
end