HBASE-22737 Add a new admin method and shell cmd to trigger the hbck chore to run (#425)
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
4829c18251
commit
35acf3cb78
|
@ -34,10 +34,15 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
|||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignsResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BypassProcedureRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BypassProcedureResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.HbckService.BlockingInterface;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ScheduleServerCrashProcedureResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignsResponse;
|
||||
|
||||
/**
|
||||
* Use {@link Connection#getHbck()} to obtain an instance of {@link Hbck} instead of
|
||||
|
@ -105,9 +110,8 @@ public class HBaseHbck implements Hbck {
|
|||
public List<Long> assigns(List<String> encodedRegionNames, boolean override)
|
||||
throws IOException {
|
||||
try {
|
||||
MasterProtos.AssignsResponse response =
|
||||
this.hbck.assigns(rpcControllerFactory.newController(),
|
||||
RequestConverter.toAssignRegionsRequest(encodedRegionNames, override));
|
||||
AssignsResponse response = this.hbck.assigns(rpcControllerFactory.newController(),
|
||||
RequestConverter.toAssignRegionsRequest(encodedRegionNames, override));
|
||||
return response.getPidList();
|
||||
} catch (ServiceException se) {
|
||||
LOG.debug(toCommaDelimitedString(encodedRegionNames), se);
|
||||
|
@ -119,9 +123,8 @@ public class HBaseHbck implements Hbck {
|
|||
public List<Long> unassigns(List<String> encodedRegionNames, boolean override)
|
||||
throws IOException {
|
||||
try {
|
||||
MasterProtos.UnassignsResponse response =
|
||||
this.hbck.unassigns(rpcControllerFactory.newController(),
|
||||
RequestConverter.toUnassignRegionsRequest(encodedRegionNames, override));
|
||||
UnassignsResponse response = this.hbck.unassigns(rpcControllerFactory.newController(),
|
||||
RequestConverter.toUnassignRegionsRequest(encodedRegionNames, override));
|
||||
return response.getPidList();
|
||||
} catch (ServiceException se) {
|
||||
LOG.debug(toCommaDelimitedString(encodedRegionNames), se);
|
||||
|
@ -137,13 +140,13 @@ public class HBaseHbck implements Hbck {
|
|||
public List<Boolean> bypassProcedure(List<Long> pids, long waitTime, boolean override,
|
||||
boolean recursive)
|
||||
throws IOException {
|
||||
MasterProtos.BypassProcedureResponse response = ProtobufUtil.call(
|
||||
new Callable<MasterProtos.BypassProcedureResponse>() {
|
||||
BypassProcedureResponse response = ProtobufUtil.call(
|
||||
new Callable<BypassProcedureResponse>() {
|
||||
@Override
|
||||
public MasterProtos.BypassProcedureResponse call() throws Exception {
|
||||
public BypassProcedureResponse call() throws Exception {
|
||||
try {
|
||||
return hbck.bypassProcedure(rpcControllerFactory.newController(),
|
||||
MasterProtos.BypassProcedureRequest.newBuilder().addAllProcId(pids).
|
||||
BypassProcedureRequest.newBuilder().addAllProcId(pids).
|
||||
setWaitTime(waitTime).setOverride(override).setRecursive(recursive).build());
|
||||
} catch (Throwable t) {
|
||||
LOG.error(pids.stream().map(i -> i.toString()).
|
||||
|
@ -159,7 +162,7 @@ public class HBaseHbck implements Hbck {
|
|||
public List<Long> scheduleServerCrashProcedures(List<ServerName> serverNames)
|
||||
throws IOException {
|
||||
try {
|
||||
MasterProtos.ScheduleServerCrashProcedureResponse response =
|
||||
ScheduleServerCrashProcedureResponse response =
|
||||
this.hbck.scheduleServerCrashProcedure(rpcControllerFactory.newController(),
|
||||
RequestConverter.toScheduleServerCrashProcedureRequest(serverNames));
|
||||
return response.getPidList();
|
||||
|
@ -171,4 +174,16 @@ public class HBaseHbck implements Hbck {
|
|||
throw new IOException(se);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean runHbckChore() throws IOException {
|
||||
try {
|
||||
RunHbckChoreResponse response = this.hbck.runHbckChore(rpcControllerFactory.newController(),
|
||||
RunHbckChoreRequest.newBuilder().build());
|
||||
return response.getRan();
|
||||
} catch (ServiceException se) {
|
||||
LOG.debug("Failed to run HBCK chore", se);
|
||||
throw new IOException(se);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -121,4 +121,12 @@ public interface Hbck extends Abortable, Closeable {
|
|||
}
|
||||
|
||||
List<Long> scheduleServerCrashProcedures(List<ServerName> serverNames) throws IOException;
|
||||
|
||||
/**
|
||||
* Request HBCK chore to run at master side.
|
||||
*
|
||||
* @return <code>true</code> if HBCK chore ran, <code>false</code> if HBCK chore already running
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
boolean runHbckChore() throws IOException;
|
||||
}
|
||||
|
|
|
@ -358,6 +358,13 @@ message IsNormalizerEnabledResponse {
|
|||
required bool enabled = 1;
|
||||
}
|
||||
|
||||
message RunHbckChoreRequest {
|
||||
}
|
||||
|
||||
message RunHbckChoreResponse {
|
||||
required bool ran = 1;
|
||||
}
|
||||
|
||||
message RunCatalogScanRequest {
|
||||
}
|
||||
|
||||
|
@ -1138,4 +1145,10 @@ service HbckService {
|
|||
/** Schedule a ServerCrashProcedure to help recover a crash server */
|
||||
rpc ScheduleServerCrashProcedure(ScheduleServerCrashProcedureRequest)
|
||||
returns(ScheduleServerCrashProcedureResponse);
|
||||
|
||||
/**
|
||||
* Request HBCK chore to run at master side.
|
||||
*/
|
||||
rpc RunHbckChore(RunHbckChoreRequest)
|
||||
returns(RunHbckChoreResponse);
|
||||
}
|
||||
|
|
|
@ -385,7 +385,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
private ClusterStatusPublisher clusterStatusPublisherChore = null;
|
||||
private SnapshotCleanerChore snapshotCleanerChore = null;
|
||||
|
||||
private HbckChecker hbckChecker;
|
||||
private HbckChore hbckChore;
|
||||
CatalogJanitor catalogJanitorChore;
|
||||
private LogCleaner logCleaner;
|
||||
private HFileCleaner hfileCleaner;
|
||||
|
@ -1109,8 +1109,8 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
getChoreService().scheduleChore(normalizerChore);
|
||||
this.catalogJanitorChore = new CatalogJanitor(this);
|
||||
getChoreService().scheduleChore(catalogJanitorChore);
|
||||
this.hbckChecker = new HbckChecker(this);
|
||||
getChoreService().scheduleChore(hbckChecker);
|
||||
this.hbckChore = new HbckChore(this);
|
||||
getChoreService().scheduleChore(hbckChore);
|
||||
this.serverManager.startChore();
|
||||
|
||||
// Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.
|
||||
|
@ -1590,7 +1590,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
choreService.cancelChore(this.hfileCleaner);
|
||||
choreService.cancelChore(this.replicationBarrierCleaner);
|
||||
choreService.cancelChore(this.snapshotCleanerChore);
|
||||
choreService.cancelChore(this.hbckChecker);
|
||||
choreService.cancelChore(this.hbckChore);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3761,7 +3761,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
return super.getWalGroupsReplicationStatus();
|
||||
}
|
||||
|
||||
public HbckChecker getHbckChecker() {
|
||||
return this.hbckChecker;
|
||||
public HbckChore getHbckChore() {
|
||||
return this.hbckChore;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,11 +47,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class HbckChecker extends ScheduledChore {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HbckChecker.class.getName());
|
||||
public class HbckChore extends ScheduledChore {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HbckChore.class.getName());
|
||||
|
||||
private static final String HBCK_CHECKER_INTERVAL = "hbase.master.hbck.checker.interval";
|
||||
private static final int DEFAULT_HBCK_CHECKER_INTERVAL = 60 * 60 * 1000;
|
||||
private static final String HBCK_CHORE_INTERVAL = "hbase.master.hbck.chore.interval";
|
||||
private static final int DEFAULT_HBCK_CHORE_INTERVAL = 60 * 60 * 1000;
|
||||
|
||||
private final MasterServices master;
|
||||
|
||||
|
@ -100,14 +100,14 @@ public class HbckChecker extends ScheduledChore {
|
|||
private volatile long checkingStartTimestamp = 0;
|
||||
private volatile long checkingEndTimestamp = 0;
|
||||
|
||||
public HbckChecker(MasterServices master) {
|
||||
super("HbckChecker-", master,
|
||||
master.getConfiguration().getInt(HBCK_CHECKER_INTERVAL, DEFAULT_HBCK_CHECKER_INTERVAL));
|
||||
public HbckChore(MasterServices master) {
|
||||
super("HbckChore-", master,
|
||||
master.getConfiguration().getInt(HBCK_CHORE_INTERVAL, DEFAULT_HBCK_CHORE_INTERVAL));
|
||||
this.master = master;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void chore() {
|
||||
protected synchronized void chore() {
|
||||
running = true;
|
||||
regionInfoMap.clear();
|
||||
orphanRegionsOnRS.clear();
|
||||
|
@ -277,6 +277,6 @@ public class HbckChecker extends ScheduledChore {
|
|||
* Used for web ui to show when the HBCK checking report generated.
|
||||
*/
|
||||
public long getCheckingEndTimestamp() {
|
||||
return this.checkingStartTimestamp;
|
||||
return this.checkingEndTimestamp;
|
||||
}
|
||||
}
|
|
@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
|
@ -68,10 +67,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
|
|||
import org.apache.hadoop.hbase.ipc.RpcServerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
|
||||
import org.apache.hadoop.hbase.master.locking.LockProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
|
||||
|
@ -91,7 +87,6 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
|||
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaUtil;
|
||||
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||
import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationException;
|
||||
|
@ -112,10 +107,8 @@ import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
|||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.PairOfSameType;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -264,6 +257,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalog
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunHbckChoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
|
||||
|
@ -2371,6 +2366,20 @@ public class MasterRpcServices extends RSRpcServices
|
|||
|
||||
// HBCK Services
|
||||
|
||||
@Override
|
||||
public RunHbckChoreResponse runHbckChore(RpcController c, RunHbckChoreRequest req)
|
||||
throws ServiceException {
|
||||
rpcPreCheck("runHbckChore");
|
||||
LOG.info("{} request HBCK chore to run", master.getClientIdAuditPrefix());
|
||||
HbckChore hbckChore = master.getHbckChore();
|
||||
boolean ran = false;
|
||||
if (!hbckChore.isRunning()) {
|
||||
hbckChore.chore();
|
||||
ran = true;
|
||||
}
|
||||
return RunHbckChoreResponse.newBuilder().setRan(ran).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Update state of the table in meta only. This is required by hbck in some situations to cleanup
|
||||
* stuck assign/ unassign regions procedures for the table.
|
||||
|
|
|
@ -27,7 +27,7 @@
|
|||
import="java.time.ZonedDateTime"
|
||||
import="java.time.format.DateTimeFormatter"
|
||||
%>
|
||||
<%@ page import="org.apache.hadoop.hbase.master.HbckChecker" %>
|
||||
<%@ page import="org.apache.hadoop.hbase.master.HbckChore" %>
|
||||
<%@ page import="org.apache.hadoop.hbase.master.HMaster" %>
|
||||
<%@ page import="org.apache.hadoop.hbase.ServerName" %>
|
||||
<%@ page import="org.apache.hadoop.hbase.util.Bytes" %>
|
||||
|
@ -38,18 +38,18 @@
|
|||
<%
|
||||
HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
|
||||
pageContext.setAttribute("pageTitle", "HBase Master HBCK Report: " + master.getServerName());
|
||||
HbckChecker hbckChecker = master.getHbckChecker();
|
||||
HbckChore hbckChore = master.getHbckChore();
|
||||
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions = null;
|
||||
Map<String, ServerName> orphanRegionsOnRS = null;
|
||||
List<String> orphanRegionsOnFS = null;
|
||||
long startTimestamp = 0;
|
||||
long endTimestamp = 0;
|
||||
if (hbckChecker != null) {
|
||||
inconsistentRegions = hbckChecker.getInconsistentRegions();
|
||||
orphanRegionsOnRS = hbckChecker.getOrphanRegionsOnRS();
|
||||
orphanRegionsOnFS = hbckChecker.getOrphanRegionsOnFS();
|
||||
startTimestamp = hbckChecker.getCheckingStartTimestamp();
|
||||
endTimestamp = hbckChecker.getCheckingEndTimestamp();
|
||||
if (hbckChore != null) {
|
||||
inconsistentRegions = hbckChore.getInconsistentRegions();
|
||||
orphanRegionsOnRS = hbckChore.getOrphanRegionsOnRS();
|
||||
orphanRegionsOnFS = hbckChore.getOrphanRegionsOnFS();
|
||||
startTimestamp = hbckChore.getCheckingStartTimestamp();
|
||||
endTimestamp = hbckChore.getCheckingEndTimestamp();
|
||||
}
|
||||
ZonedDateTime zdt = ZonedDateTime.ofInstant(Instant.ofEpochMilli(startTimestamp),
|
||||
ZoneId.systemDefault());
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
|
|||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
|
||||
|
@ -237,6 +238,20 @@ public class TestHbck {
|
|||
waitOnPids(pids);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunHbckChore() throws Exception {
|
||||
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
|
||||
long endTimestamp = master.getHbckChore().getCheckingEndTimestamp();
|
||||
Hbck hbck = getHbck();
|
||||
boolean ran = false;
|
||||
while (!ran) {
|
||||
ran = hbck.runHbckChore();
|
||||
if (ran) {
|
||||
assertTrue(master.getHbckChore().getCheckingEndTimestamp() > endTimestamp);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class FailingSplitAfterMetaUpdatedMasterObserver
|
||||
implements MasterCoprocessor, MasterObserver {
|
||||
public volatile CountDownLatch latch;
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.master.HbckChecker;
|
||||
import org.apache.hadoop.hbase.master.HbckChore;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -43,19 +43,19 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@Category({ MasterTests.class, MediumTests.class })
|
||||
public class TestHbckChecker extends TestAssignmentManagerBase {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestHbckChecker.class);
|
||||
public class TestHbckChore extends TestAssignmentManagerBase {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestHbckChore.class);
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestHbckChecker.class);
|
||||
HBaseClassTestRule.forClass(TestHbckChore.class);
|
||||
|
||||
private HbckChecker hbckChecker;
|
||||
private HbckChore hbckChore;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
hbckChecker = new HbckChecker(master);
|
||||
hbckChore = new HbckChore(master);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -65,9 +65,9 @@ public class TestHbckChecker extends TestAssignmentManagerBase {
|
|||
List<ServerName> serverNames = master.getServerManager().getOnlineServersList();
|
||||
assertEquals(NSERVERS, serverNames.size());
|
||||
|
||||
hbckChecker.choreForTesting();
|
||||
hbckChore.choreForTesting();
|
||||
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
|
||||
hbckChecker.getInconsistentRegions();
|
||||
hbckChore.getInconsistentRegions();
|
||||
|
||||
// Test for case1: Master thought this region opened, but no regionserver reported it.
|
||||
assertTrue(inconsistentRegions.containsKey(metaRegionName));
|
||||
|
@ -79,8 +79,8 @@ public class TestHbckChecker extends TestAssignmentManagerBase {
|
|||
|
||||
// Reported right region location. Then not in problematic regions.
|
||||
am.reportOnlineRegions(locationInMeta, Collections.singleton(metaRegionNameAsBytes));
|
||||
hbckChecker.choreForTesting();
|
||||
inconsistentRegions = hbckChecker.getInconsistentRegions();
|
||||
hbckChore.choreForTesting();
|
||||
inconsistentRegions = hbckChore.getInconsistentRegions();
|
||||
assertFalse(inconsistentRegions.containsKey(metaRegionName));
|
||||
}
|
||||
|
||||
|
@ -97,9 +97,9 @@ public class TestHbckChecker extends TestAssignmentManagerBase {
|
|||
assertEquals(NSERVERS, serverNames.size());
|
||||
|
||||
// Test for case1: Master thought this region opened, but no regionserver reported it.
|
||||
hbckChecker.choreForTesting();
|
||||
hbckChore.choreForTesting();
|
||||
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
|
||||
hbckChecker.getInconsistentRegions();
|
||||
hbckChore.getInconsistentRegions();
|
||||
assertTrue(inconsistentRegions.containsKey(regionName));
|
||||
Pair<ServerName, List<ServerName>> pair = inconsistentRegions.get(regionName);
|
||||
ServerName locationInMeta = pair.getFirst();
|
||||
|
@ -113,8 +113,8 @@ public class TestHbckChecker extends TestAssignmentManagerBase {
|
|||
final ServerName anotherServer =
|
||||
serverNames.stream().filter(s -> !s.equals(tempLocationInMeta)).findFirst().get();
|
||||
am.reportOnlineRegions(anotherServer, Collections.singleton(hri.getRegionName()));
|
||||
hbckChecker.choreForTesting();
|
||||
inconsistentRegions = hbckChecker.getInconsistentRegions();
|
||||
hbckChore.choreForTesting();
|
||||
inconsistentRegions = hbckChore.getInconsistentRegions();
|
||||
assertTrue(inconsistentRegions.containsKey(regionName));
|
||||
pair = inconsistentRegions.get(regionName);
|
||||
locationInMeta = pair.getFirst();
|
||||
|
@ -125,8 +125,8 @@ public class TestHbckChecker extends TestAssignmentManagerBase {
|
|||
|
||||
// Test for case3: More than one regionservers reported opened this region.
|
||||
am.reportOnlineRegions(locationInMeta, Collections.singleton(hri.getRegionName()));
|
||||
hbckChecker.choreForTesting();
|
||||
inconsistentRegions = hbckChecker.getInconsistentRegions();
|
||||
hbckChore.choreForTesting();
|
||||
inconsistentRegions = hbckChore.getInconsistentRegions();
|
||||
assertTrue(inconsistentRegions.containsKey(regionName));
|
||||
pair = inconsistentRegions.get(regionName);
|
||||
locationInMeta = pair.getFirst();
|
||||
|
@ -137,8 +137,8 @@ public class TestHbckChecker extends TestAssignmentManagerBase {
|
|||
|
||||
// Reported right region location. Then not in problematic regions.
|
||||
am.reportOnlineRegions(anotherServer, Collections.EMPTY_SET);
|
||||
hbckChecker.choreForTesting();
|
||||
inconsistentRegions = hbckChecker.getInconsistentRegions();
|
||||
hbckChore.choreForTesting();
|
||||
inconsistentRegions = hbckChore.getInconsistentRegions();
|
||||
assertFalse(inconsistentRegions.containsKey(regionName));
|
||||
}
|
||||
}
|
|
@ -37,6 +37,7 @@ module Hbase
|
|||
@connection = connection
|
||||
# Java Admin instance
|
||||
@admin = @connection.getAdmin
|
||||
@hbck = @connection.getHbck
|
||||
@conf = @connection.getConfiguration
|
||||
end
|
||||
|
||||
|
@ -266,6 +267,12 @@ module Hbase
|
|||
@admin.isMasterInMaintenanceMode
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Request HBCK chore to run
|
||||
def hbck_chore_run
|
||||
@hbck.runHbckChore
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Request a scan of the catalog table (for garbage collection)
|
||||
# Returns an int signifying the number of entries cleaned
|
||||
|
|
|
@ -344,6 +344,7 @@ Shell.load_command_group(
|
|||
unassign
|
||||
zk_dump
|
||||
wal_roll
|
||||
hbck_chore_run
|
||||
catalogjanitor_run
|
||||
catalogjanitor_switch
|
||||
catalogjanitor_enabled
|
||||
|
|
|
@ -0,0 +1,38 @@
|
|||
#
|
||||
# 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 HbckChoreRun < Command
|
||||
def help
|
||||
<<-EOF
|
||||
Request HBCK chore to run at master side. It will try to find the orphan
|
||||
regions on RegionServer or FileSystem and find the inconsistent regions.
|
||||
You can check the HBCK report at Master web UI.
|
||||
|
||||
hbase> hbck_chore_run
|
||||
|
||||
EOF
|
||||
end
|
||||
|
||||
def command
|
||||
admin.hbck_chore_run
|
||||
end
|
||||
end
|
||||
end
|
||||
end
|
|
@ -73,6 +73,10 @@ module Hbase
|
|||
# Some text which isn't in the simple output
|
||||
assert output.include? 'regionsInTransition'
|
||||
end
|
||||
|
||||
define_test 'hbck_chore_run' do
|
||||
command(:hbck_chore_run)
|
||||
end
|
||||
end
|
||||
|
||||
# Simple administration methods tests
|
||||
|
|
Loading…
Reference in New Issue