HBASE-27095 HbckChore should produce a report

In #4470 for HBASE-26192, it was noted that the HbckChore is kind of a pain to use and test
because it maintains a bunch of local state. By contract, the CatalogJanitorChore makes a nice
self-contained report. Let's update HbckChore to do the same.

Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
Nick Dimiduk 2022-06-10 15:47:08 +02:00 committed by GitHub
parent b7d7be0156
commit cc2af766d6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 304 additions and 279 deletions

View File

@ -130,6 +130,7 @@ import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner; import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner; import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
import org.apache.hadoop.hbase.master.cleaner.SnapshotCleanerChore; import org.apache.hadoop.hbase.master.cleaner.SnapshotCleanerChore;
import org.apache.hadoop.hbase.master.hbck.HbckChore;
import org.apache.hadoop.hbase.master.http.MasterDumpServlet; import org.apache.hadoop.hbase.master.http.MasterDumpServlet;
import org.apache.hadoop.hbase.master.http.MasterRedirectServlet; import org.apache.hadoop.hbase.master.http.MasterRedirectServlet;
import org.apache.hadoop.hbase.master.http.MasterStatusServlet; import org.apache.hadoop.hbase.master.http.MasterStatusServlet;

View File

@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode; import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.hbck.HbckChore;
import org.apache.hadoop.hbase.master.janitor.MetaFixer; import org.apache.hadoop.hbase.master.janitor.MetaFixer;
import org.apache.hadoop.hbase.master.locking.LockProcedure; import org.apache.hadoop.hbase.master.locking.LockProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;

View File

@ -15,16 +15,16 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.master; package org.apache.hadoop.hbase.master.hbck;
import java.io.IOException; import java.io.IOException;
import java.time.Instant;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor;
@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HbckRegionInfo; import org.apache.hadoop.hbase.util.HbckRegionInfo;
@ -55,51 +57,14 @@ public class HbckChore extends ScheduledChore {
private final MasterServices master; private final MasterServices master;
/** /**
* This map contains the state of all hbck items. It maps from encoded region name to * Saved report from last time this chore ran. Check its date.
* HbckRegionInfo structure. The information contained in HbckRegionInfo is used to detect and
* correct consistency (hdfs/meta/deployment) problems.
*/ */
private final Map<String, HbckRegionInfo> regionInfoMap = new HashMap<>(); private volatile HbckReport lastReport = null;
private final Set<String> disabledTableRegions = new HashSet<>();
private final Set<String> splitParentRegions = new HashSet<>();
/**
* The regions only opened on RegionServers, but no region info in meta.
*/
private final Map<String, ServerName> orphanRegionsOnRS = new HashMap<>();
/**
* The regions have directory on FileSystem, but no region info in meta.
*/
private final Map<String, Path> orphanRegionsOnFS = new HashMap<>();
/**
* The inconsistent regions. There are three case: case 1. Master thought this region opened, but
* no regionserver reported it. case 2. Master thought this region opened on Server1, but
* regionserver reported Server2 case 3. More than one regionservers reported opened this region
*/
private final Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
new HashMap<>();
/**
* The "snapshot" is used to save the last round's HBCK checking report.
*/
private final Map<String, ServerName> orphanRegionsOnRSSnapshot = new HashMap<>();
private final Map<String, Path> orphanRegionsOnFSSnapshot = new HashMap<>();
private final Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegionsSnapshot =
new HashMap<>();
/**
* The "snapshot" may be changed after checking. And this checking report "snapshot" may be
* accessed by web ui. Use this rwLock to synchronize.
*/
ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
/** /**
* When running, the "snapshot" may be changed when this round's checking finish. * When running, the "snapshot" may be changed when this round's checking finish.
*/ */
private volatile boolean running = false; private volatile boolean running = false;
private volatile long checkingStartTimestamp = 0;
private volatile long checkingEndTimestamp = 0;
private boolean disabled = false; private boolean disabled = false;
@ -115,39 +80,47 @@ public class HbckChore extends ScheduledChore {
} }
} }
/**
* @return Returns last published Report that comes of last successful execution of this chore.
*/
public HbckReport getLastReport() {
return lastReport;
}
@Override @Override
protected synchronized void chore() { protected synchronized void chore() {
if (isDisabled() || isRunning()) { if (isDisabled() || isRunning()) {
LOG.warn("hbckChore is either disabled or is already running. Can't run the chore"); LOG.warn("hbckChore is either disabled or is already running. Can't run the chore");
return; return;
} }
regionInfoMap.clear();
disabledTableRegions.clear();
splitParentRegions.clear();
orphanRegionsOnRS.clear();
orphanRegionsOnFS.clear();
inconsistentRegions.clear();
checkingStartTimestamp = EnvironmentEdgeManager.currentTime();
running = true; running = true;
final HbckReport report = new HbckReport();
report.setCheckingStartTimestamp(Instant.ofEpochMilli(EnvironmentEdgeManager.currentTime()));
try { try {
loadRegionsFromInMemoryState(); loadRegionsFromInMemoryState(report);
loadRegionsFromRSReport(); loadRegionsFromRSReport(report);
try { try {
loadRegionsFromFS(scanForMergedParentRegions()); loadRegionsFromFS(scanForMergedParentRegions(), report);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Failed to load the regions from filesystem", e); LOG.warn("Failed to load the regions from filesystem", e);
} }
saveCheckResultToSnapshot();
} catch (Throwable t) { } catch (Throwable t) {
LOG.warn("Unexpected", t); LOG.warn("Unexpected", t);
} }
report.setCheckingEndTimestamp(Instant.ofEpochMilli(EnvironmentEdgeManager.currentTime()));
this.lastReport = report;
running = false; running = false;
updateAssignmentManagerMetrics(); updateAssignmentManagerMetrics(report);
} }
/**
* Request execution of this chore's action.
* @return {@code true} if the chore was executed, {@code false} if the chore is disabled or
* already running.
*/
public boolean runChore() {
// This function does the sanity checks of making sure the chore is not run when it is // This function does the sanity checks of making sure the chore is not run when it is
// disabled or when it's already running. It returns whether the chore was actually run or not. // disabled or when it's already running. It returns whether the chore was actually run or not.
protected boolean runChore() {
if (isDisabled() || isRunning()) { if (isDisabled() || isRunning()) {
if (isDisabled()) { if (isDisabled()) {
LOG.warn("hbck chore is disabled! Set " + HBCK_CHORE_INTERVAL + " > 0 to enable it."); LOG.warn("hbck chore is disabled! Set " + HBCK_CHORE_INTERVAL + " > 0 to enable it.");
@ -168,25 +141,6 @@ public class HbckChore extends ScheduledChore {
return this.disabled; return this.disabled;
} }
private void saveCheckResultToSnapshot() {
// Need synchronized here, as this "snapshot" may be access by web ui.
rwLock.writeLock().lock();
try {
orphanRegionsOnRSSnapshot.clear();
orphanRegionsOnRS.entrySet()
.forEach(e -> orphanRegionsOnRSSnapshot.put(e.getKey(), e.getValue()));
orphanRegionsOnFSSnapshot.clear();
orphanRegionsOnFS.entrySet()
.forEach(e -> orphanRegionsOnFSSnapshot.put(e.getKey(), e.getValue()));
inconsistentRegionsSnapshot.clear();
inconsistentRegions.entrySet()
.forEach(e -> inconsistentRegionsSnapshot.put(e.getKey(), e.getValue()));
checkingEndTimestamp = EnvironmentEdgeManager.currentTime();
} finally {
rwLock.writeLock().unlock();
}
}
/** /**
* Scan hbase:meta to get set of merged parent regions, this is a very heavy scan. * Scan hbase:meta to get set of merged parent regions, this is a very heavy scan.
* @return Return generated {@link HashSet} * @return Return generated {@link HashSet}
@ -209,7 +163,7 @@ public class HbckChore extends ScheduledChore {
return mergedParentRegions; return mergedParentRegions;
} }
private void loadRegionsFromInMemoryState() { private void loadRegionsFromInMemoryState(final HbckReport report) {
List<RegionState> regionStates = List<RegionState> regionStates =
master.getAssignmentManager().getRegionStates().getRegionStates(); master.getAssignmentManager().getRegionStates().getRegionStates();
for (RegionState regionState : regionStates) { for (RegionState regionState : regionStates) {
@ -217,18 +171,19 @@ public class HbckChore extends ScheduledChore {
if ( if (
master.getTableStateManager().isTableState(regionInfo.getTable(), TableState.State.DISABLED) master.getTableStateManager().isTableState(regionInfo.getTable(), TableState.State.DISABLED)
) { ) {
disabledTableRegions.add(regionInfo.getRegionNameAsString()); report.getDisabledTableRegions().add(regionInfo.getRegionNameAsString());
} }
// Check both state and regioninfo for split status, see HBASE-26383 // Check both state and regioninfo for split status, see HBASE-26383
if (regionState.isSplit() || regionInfo.isSplit()) { if (regionState.isSplit() || regionInfo.isSplit()) {
splitParentRegions.add(regionInfo.getRegionNameAsString()); report.getSplitParentRegions().add(regionInfo.getRegionNameAsString());
} }
HbckRegionInfo.MetaEntry metaEntry = new HbckRegionInfo.MetaEntry(regionInfo, HbckRegionInfo.MetaEntry metaEntry = new HbckRegionInfo.MetaEntry(regionInfo,
regionState.getServerName(), regionState.getStamp()); regionState.getServerName(), regionState.getStamp());
regionInfoMap.put(regionInfo.getEncodedName(), new HbckRegionInfo(metaEntry)); report.getRegionInfoMap().put(regionInfo.getEncodedName(), new HbckRegionInfo(metaEntry));
} }
LOG.info("Loaded {} regions ({} disabled, {} split parents) from in-memory state", LOG.info("Loaded {} regions ({} disabled, {} split parents) from in-memory state",
regionStates.size(), disabledTableRegions.size(), splitParentRegions.size()); regionStates.size(), report.getDisabledTableRegions().size(),
report.getSplitParentRegions().size());
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
Map<RegionState.State, Integer> stateCountMap = new HashMap<>(); Map<RegionState.State, Integer> stateCountMap = new HashMap<>();
for (RegionState regionState : regionStates) { for (RegionState regionState : regionStates) {
@ -246,22 +201,23 @@ public class HbckChore extends ScheduledChore {
} }
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
for (RegionState regionState : regionStates) { for (RegionState regionState : regionStates) {
LOG.trace("{}: {}, serverName=", regionState.getRegion(), regionState.getState(), LOG.trace("{}: {}, serverName={}", regionState.getRegion(), regionState.getState(),
regionState.getServerName()); regionState.getServerName());
} }
} }
} }
private void loadRegionsFromRSReport() { private void loadRegionsFromRSReport(final HbckReport report) {
int numRegions = 0; int numRegions = 0;
Map<ServerName, Set<byte[]>> rsReports = master.getAssignmentManager().getRSReports(); Map<ServerName, Set<byte[]>> rsReports = master.getAssignmentManager().getRSReports();
for (Map.Entry<ServerName, Set<byte[]>> entry : rsReports.entrySet()) { for (Map.Entry<ServerName, Set<byte[]>> entry : rsReports.entrySet()) {
ServerName serverName = entry.getKey(); ServerName serverName = entry.getKey();
for (byte[] regionName : entry.getValue()) { for (byte[] regionName : entry.getValue()) {
String encodedRegionName = RegionInfo.encodeRegionName(regionName); String encodedRegionName = RegionInfo.encodeRegionName(regionName);
HbckRegionInfo hri = regionInfoMap.get(encodedRegionName); HbckRegionInfo hri = report.getRegionInfoMap().get(encodedRegionName);
if (hri == null) { if (hri == null) {
orphanRegionsOnRS.put(RegionInfo.getRegionNameAsString(regionName), serverName); report.getOrphanRegionsOnRS().put(RegionInfo.getRegionNameAsString(regionName),
serverName);
continue; continue;
} }
hri.addServer(hri.getMetaEntry(), serverName); hri.addServer(hri.getMetaEntry(), serverName);
@ -269,9 +225,9 @@ public class HbckChore extends ScheduledChore {
numRegions += entry.getValue().size(); numRegions += entry.getValue().size();
} }
LOG.info("Loaded {} regions from {} regionservers' reports and found {} orphan regions", LOG.info("Loaded {} regions from {} regionservers' reports and found {} orphan regions",
numRegions, rsReports.size(), orphanRegionsOnRS.size()); numRegions, rsReports.size(), report.getOrphanRegionsOnRS().size());
for (Map.Entry<String, HbckRegionInfo> entry : regionInfoMap.entrySet()) { for (Map.Entry<String, HbckRegionInfo> entry : report.getRegionInfoMap().entrySet()) {
HbckRegionInfo hri = entry.getValue(); HbckRegionInfo hri = entry.getValue();
ServerName locationInMeta = hri.getMetaEntry().getRegionServer(); ServerName locationInMeta = hri.getMetaEntry().getRegionServer();
if (locationInMeta == null) { if (locationInMeta == null) {
@ -279,29 +235,30 @@ public class HbckChore extends ScheduledChore {
} }
if (hri.getDeployedOn().size() == 0) { if (hri.getDeployedOn().size() == 0) {
// skip the offline region which belong to disabled table. // skip the offline region which belong to disabled table.
if (disabledTableRegions.contains(hri.getRegionNameAsString())) { if (report.getDisabledTableRegions().contains(hri.getRegionNameAsString())) {
continue; continue;
} }
// skip the split parent regions // skip the split parent regions
if (splitParentRegions.contains(hri.getRegionNameAsString())) { if (report.getSplitParentRegions().contains(hri.getRegionNameAsString())) {
continue; continue;
} }
// Master thought this region opened, but no regionserver reported it. // Master thought this region opened, but no regionserver reported it.
inconsistentRegions.put(hri.getRegionNameAsString(), report.getInconsistentRegions().put(hri.getRegionNameAsString(),
new Pair<>(locationInMeta, new LinkedList<>())); new Pair<>(locationInMeta, new LinkedList<>()));
} else if (hri.getDeployedOn().size() > 1) { } else if (hri.getDeployedOn().size() > 1) {
// More than one regionserver reported opened this region // More than one regionserver reported opened this region
inconsistentRegions.put(hri.getRegionNameAsString(), report.getInconsistentRegions().put(hri.getRegionNameAsString(),
new Pair<>(locationInMeta, hri.getDeployedOn())); new Pair<>(locationInMeta, hri.getDeployedOn()));
} else if (!hri.getDeployedOn().get(0).equals(locationInMeta)) { } else if (!hri.getDeployedOn().get(0).equals(locationInMeta)) {
// Master thought this region opened on Server1, but regionserver reported Server2 // Master thought this region opened on Server1, but regionserver reported Server2
inconsistentRegions.put(hri.getRegionNameAsString(), report.getInconsistentRegions().put(hri.getRegionNameAsString(),
new Pair<>(locationInMeta, hri.getDeployedOn())); new Pair<>(locationInMeta, hri.getDeployedOn()));
} }
} }
} }
private void loadRegionsFromFS(final HashSet<String> mergedParentRegions) throws IOException { private void loadRegionsFromFS(final HashSet<String> mergedParentRegions, final HbckReport report)
throws IOException {
Path rootDir = master.getMasterFileSystem().getRootDir(); Path rootDir = master.getMasterFileSystem().getRootDir();
FileSystem fs = master.getMasterFileSystem().getFileSystem(); FileSystem fs = master.getMasterFileSystem().getFileSystem();
@ -315,27 +272,27 @@ public class HbckChore extends ScheduledChore {
LOG.warn("Failed get of encoded name from {}", regionDir); LOG.warn("Failed get of encoded name from {}", regionDir);
continue; continue;
} }
HbckRegionInfo hri = regionInfoMap.get(encodedRegionName); HbckRegionInfo hri = report.getRegionInfoMap().get(encodedRegionName);
// If it is not in in-memory database and not a merged region, // If it is not in in-memory database and not a merged region,
// report it as an orphan region. // report it as an orphan region.
if (hri == null && !mergedParentRegions.contains(encodedRegionName)) { if (hri == null && !mergedParentRegions.contains(encodedRegionName)) {
orphanRegionsOnFS.put(encodedRegionName, regionDir); report.getOrphanRegionsOnFS().put(encodedRegionName, regionDir);
continue; continue;
} }
} }
numRegions += regionDirs.size(); numRegions += regionDirs.size();
} }
LOG.info("Loaded {} tables {} regions from filesystem and found {} orphan regions", LOG.info("Loaded {} tables {} regions from filesystem and found {} orphan regions",
tableDirs.size(), numRegions, orphanRegionsOnFS.size()); tableDirs.size(), numRegions, report.getOrphanRegionsOnFS().size());
} }
private void updateAssignmentManagerMetrics() { private void updateAssignmentManagerMetrics(final HbckReport report) {
master.getAssignmentManager().getAssignmentManagerMetrics() master.getAssignmentManager().getAssignmentManagerMetrics()
.updateOrphanRegionsOnRs(getOrphanRegionsOnRS().size()); .updateOrphanRegionsOnRs(report.getOrphanRegionsOnRS().size());
master.getAssignmentManager().getAssignmentManagerMetrics() master.getAssignmentManager().getAssignmentManagerMetrics()
.updateOrphanRegionsOnFs(getOrphanRegionsOnFS().size()); .updateOrphanRegionsOnFs(report.getOrphanRegionsOnFS().size());
master.getAssignmentManager().getAssignmentManagerMetrics() master.getAssignmentManager().getAssignmentManagerMetrics()
.updateInconsistentRegions(getInconsistentRegions().size()); .updateInconsistentRegions(report.getInconsistentRegions().size());
} }
/** /**
@ -344,62 +301,4 @@ public class HbckChore extends ScheduledChore {
public boolean isRunning() { public boolean isRunning() {
return running; return running;
} }
/**
* @return the regions only opened on RegionServers, but no region info in meta.
*/
public Map<String, ServerName> getOrphanRegionsOnRS() {
// Need synchronized here, as this "snapshot" may be changed after checking.
rwLock.readLock().lock();
try {
return this.orphanRegionsOnRSSnapshot;
} finally {
rwLock.readLock().unlock();
}
}
/**
* @return the regions have directory on FileSystem, but no region info in meta.
*/
public Map<String, Path> getOrphanRegionsOnFS() {
// Need synchronized here, as this "snapshot" may be changed after checking.
rwLock.readLock().lock();
try {
return this.orphanRegionsOnFSSnapshot;
} finally {
rwLock.readLock().unlock();
}
}
/**
* Found the inconsistent regions. There are three case: case 1. Master thought this region
* opened, but no regionserver reported it. case 2. Master thought this region opened on Server1,
* but regionserver reported Server2 case 3. More than one regionservers reported opened this
* region
* @return the map of inconsistent regions. Key is the region name. Value is a pair of location in
* meta and the regionservers which reported opened this region.
*/
public Map<String, Pair<ServerName, List<ServerName>>> getInconsistentRegions() {
// Need synchronized here, as this "snapshot" may be changed after checking.
rwLock.readLock().lock();
try {
return this.inconsistentRegionsSnapshot;
} finally {
rwLock.readLock().unlock();
}
}
/**
* Used for web ui to show when the HBCK checking started.
*/
public long getCheckingStartTimestamp() {
return this.checkingStartTimestamp;
}
/**
* Used for web ui to show when the HBCK checking report generated.
*/
public long getCheckingEndTimestamp() {
return this.checkingEndTimestamp;
}
} }

View File

@ -0,0 +1,110 @@
/*
* 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.
*/
package org.apache.hadoop.hbase.master.hbck;
import java.time.Instant;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.HbckRegionInfo;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The result of an {@link HbckChore} execution.
*/
@InterfaceAudience.Private
public class HbckReport {
private final Map<String, HbckRegionInfo> regionInfoMap = new HashMap<>();
private final Set<String> disabledTableRegions = new HashSet<>();
private final Set<String> splitParentRegions = new HashSet<>();
private final Map<String, ServerName> orphanRegionsOnRS = new HashMap<>();
private final Map<String, Path> orphanRegionsOnFS = new HashMap<>();
private final Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
new HashMap<>();
private Instant checkingStartTimestamp = null;
private Instant checkingEndTimestamp = null;
/**
* Used for web ui to show when the HBCK checking started.
*/
public Instant getCheckingStartTimestamp() {
return checkingStartTimestamp;
}
public void setCheckingStartTimestamp(Instant checkingStartTimestamp) {
this.checkingStartTimestamp = checkingStartTimestamp;
}
/**
* Used for web ui to show when the HBCK checking report generated.
*/
public Instant getCheckingEndTimestamp() {
return checkingEndTimestamp;
}
public void setCheckingEndTimestamp(Instant checkingEndTimestamp) {
this.checkingEndTimestamp = checkingEndTimestamp;
}
/**
* This map contains the state of all hbck items. It maps from encoded region name to
* HbckRegionInfo structure. The information contained in HbckRegionInfo is used to detect and
* correct consistency (hdfs/meta/deployment) problems.
*/
public Map<String, HbckRegionInfo> getRegionInfoMap() {
return regionInfoMap;
}
public Set<String> getDisabledTableRegions() {
return disabledTableRegions;
}
public Set<String> getSplitParentRegions() {
return splitParentRegions;
}
/**
* The regions only opened on RegionServers, but no region info in meta.
*/
public Map<String, ServerName> getOrphanRegionsOnRS() {
return orphanRegionsOnRS;
}
/**
* The regions have directory on FileSystem, but no region info in meta.
*/
public Map<String, Path> getOrphanRegionsOnFS() {
return orphanRegionsOnFS;
}
/**
* The inconsistent regions. There are three case: case 1. Master thought this region opened, but
* no regionserver reported it. case 2. Master thought this region opened on Server1, but
* regionserver reported Server2 case 3. More than one regionservers reported opened this region
*/
public Map<String, Pair<ServerName, List<ServerName>>> getInconsistentRegions() {
return inconsistentRegions;
}
}

View File

@ -85,7 +85,7 @@ public class CatalogJanitor extends ScheduledChore {
* Saved report from last hbase:meta scan to completion. May be stale if having trouble completing * Saved report from last hbase:meta scan to completion. May be stale if having trouble completing
* scan. Check its date. * scan. Check its date.
*/ */
private volatile Report lastReport; private volatile CatalogJanitorReport lastReport;
public CatalogJanitor(final MasterServices services) { public CatalogJanitor(final MasterServices services) {
super("CatalogJanitor-" + services.getServerName().toShortString(), services, super("CatalogJanitor-" + services.getServerName().toShortString(), services,
@ -227,10 +227,10 @@ public class CatalogJanitor extends ScheduledChore {
/** /**
* Scan hbase:meta. * Scan hbase:meta.
* @return Return generated {@link Report} * @return Return generated {@link CatalogJanitorReport}
*/ */
// will be override in tests. // will be override in tests.
protected Report scanForReport() throws IOException { protected CatalogJanitorReport scanForReport() throws IOException {
ReportMakingVisitor visitor = new ReportMakingVisitor(this.services); ReportMakingVisitor visitor = new ReportMakingVisitor(this.services);
// Null tablename means scan all of meta. // Null tablename means scan all of meta.
MetaTableAccessor.scanMetaForTableRegions(this.services.getConnection(), visitor, null); MetaTableAccessor.scanMetaForTableRegions(this.services.getConnection(), visitor, null);
@ -240,7 +240,7 @@ public class CatalogJanitor extends ScheduledChore {
/** /**
* @return Returns last published Report that comes of last successful scan of hbase:meta. * @return Returns last published Report that comes of last successful scan of hbase:meta.
*/ */
public Report getLastReport() { public CatalogJanitorReport getLastReport() {
return this.lastReport; return this.lastReport;
} }
@ -493,7 +493,7 @@ public class CatalogJanitor extends ScheduledChore {
t.put(p); t.put(p);
} }
MetaTableAccessor.scanMetaForTableRegions(connection, visitor, null); MetaTableAccessor.scanMetaForTableRegions(connection, visitor, null);
Report report = visitor.getReport(); CatalogJanitorReport report = visitor.getReport();
LOG.info(report != null ? report.toString() : "empty"); LOG.info(report != null ? report.toString() : "empty");
} }
} }

View File

@ -34,7 +34,7 @@ import org.apache.yetus.audience.InterfaceAudience;
* Report made by ReportMakingVisitor * Report made by ReportMakingVisitor
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class Report { public class CatalogJanitorReport {
private final long now = EnvironmentEdgeManager.currentTime(); private final long now = EnvironmentEdgeManager.currentTime();
// Keep Map of found split parents. These are candidates for cleanup. // Keep Map of found split parents. These are candidates for cleanup.

View File

@ -76,7 +76,7 @@ public class MetaFixer {
} }
public void fix() throws IOException { public void fix() throws IOException {
Report report = this.masterServices.getCatalogJanitor().getLastReport(); CatalogJanitorReport report = this.masterServices.getCatalogJanitor().getLastReport();
if (report == null) { if (report == null) {
LOG.info("CatalogJanitor has not generated a report yet; run 'catalogjanitor_run' in " LOG.info("CatalogJanitor has not generated a report yet; run 'catalogjanitor_run' in "
+ "shell or wait until CatalogJanitor chore runs."); + "shell or wait until CatalogJanitor chore runs.");
@ -93,7 +93,7 @@ public class MetaFixer {
* If hole, it papers it over by adding a region in the filesystem and to hbase:meta. Does not * If hole, it papers it over by adding a region in the filesystem and to hbase:meta. Does not
* assign. * assign.
*/ */
void fixHoles(Report report) { void fixHoles(CatalogJanitorReport report) {
final List<Pair<RegionInfo, RegionInfo>> holes = report.getHoles(); final List<Pair<RegionInfo, RegionInfo>> holes = report.getHoles();
if (holes.isEmpty()) { if (holes.isEmpty()) {
LOG.info("CatalogJanitor Report contains no holes to fix. Skipping."); LOG.info("CatalogJanitor Report contains no holes to fix. Skipping.");
@ -229,7 +229,7 @@ public class MetaFixer {
/** /**
* Fix overlaps noted in CJ consistency report. * Fix overlaps noted in CJ consistency report.
*/ */
List<Long> fixOverlaps(Report report) throws IOException { List<Long> fixOverlaps(CatalogJanitorReport report) throws IOException {
List<Long> pidList = new ArrayList<>(); List<Long> pidList = new ArrayList<>();
for (Set<RegionInfo> regions : calculateMerges(maxMergeCount, report.getOverlaps())) { for (Set<RegionInfo> regions : calculateMerges(maxMergeCount, report.getOverlaps())) {
RegionInfo[] regionsArray = regions.toArray(new RegionInfo[] {}); RegionInfo[] regionsArray = regions.toArray(new RegionInfo[] {});

View File

@ -53,7 +53,7 @@ class ReportMakingVisitor implements MetaTableAccessor.CloseableVisitor {
/** /**
* Report is not done until after the close has been called. * Report is not done until after the close has been called.
*/ */
private Report report = new Report(); private CatalogJanitorReport report = new CatalogJanitorReport();
/** /**
* RegionInfo from previous row. * RegionInfo from previous row.
@ -76,7 +76,7 @@ class ReportMakingVisitor implements MetaTableAccessor.CloseableVisitor {
/** /**
* Do not call until after {@link #close()}. Will throw a {@link RuntimeException} if you do. * Do not call until after {@link #close()}. Will throw a {@link RuntimeException} if you do.
*/ */
Report getReport() { CatalogJanitorReport getReport() {
if (!this.closed) { if (!this.closed) {
throw new RuntimeException("Report not ready until after close()"); throw new RuntimeException("Report not ready until after close()");
} }

View File

@ -20,7 +20,6 @@
<%@ page contentType="text/html;charset=UTF-8" <%@ page contentType="text/html;charset=UTF-8"
import="java.time.Instant" import="java.time.Instant"
import="java.time.ZoneId" import="java.time.ZoneId"
import="java.util.Date"
import="java.util.List" import="java.util.List"
import="java.util.Map" import="java.util.Map"
import="java.util.stream.Collectors" import="java.util.stream.Collectors"
@ -29,14 +28,17 @@
%> %>
<%@ page import="org.apache.hadoop.fs.Path" %> <%@ page import="org.apache.hadoop.fs.Path" %>
<%@ page import="org.apache.hadoop.hbase.client.RegionInfo" %> <%@ page import="org.apache.hadoop.hbase.client.RegionInfo" %>
<%@ page import="org.apache.hadoop.hbase.master.HbckChore" %> <%@ page import="org.apache.hadoop.hbase.master.hbck.HbckChore" %>
<%@ page import="org.apache.hadoop.hbase.master.hbck.HbckReport" %>
<%@ page import="org.apache.hadoop.hbase.master.HMaster" %> <%@ page import="org.apache.hadoop.hbase.master.HMaster" %>
<%@ page import="org.apache.hadoop.hbase.master.ServerManager" %> <%@ page import="org.apache.hadoop.hbase.master.ServerManager" %>
<%@ page import="org.apache.hadoop.hbase.ServerName" %> <%@ page import="org.apache.hadoop.hbase.ServerName" %>
<%@ page import="org.apache.hadoop.hbase.util.Bytes" %> <%@ page import="org.apache.hadoop.hbase.util.Bytes" %>
<%@ page import="org.apache.hadoop.hbase.util.Pair" %> <%@ page import="org.apache.hadoop.hbase.util.Pair" %>
<%@ page import="org.apache.hadoop.hbase.master.janitor.CatalogJanitor" %> <%@ page import="org.apache.hadoop.hbase.master.janitor.CatalogJanitor" %>
<%@ page import="org.apache.hadoop.hbase.master.janitor.Report" %> <%@ page import="org.apache.hadoop.hbase.master.janitor.CatalogJanitorReport" %>
<%@ page import="java.util.Optional" %>
<%@ page import="org.apache.hadoop.hbase.util.EnvironmentEdgeManager" %>
<% <%
final String cacheParameterValue = request.getParameter("cache"); final String cacheParameterValue = request.getParameter("cache");
final HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER); final HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
@ -55,26 +57,19 @@
} }
} }
HbckChore hbckChore = master.getHbckChore(); HbckChore hbckChore = master.getHbckChore();
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions = null; HbckReport hbckReport = hbckChore == null ? null : hbckChore.getLastReport();
Map<String, ServerName> orphanRegionsOnRS = null; String hbckReportStartTime = Optional.ofNullable(hbckReport)
Map<String, Path> orphanRegionsOnFS = null; .map(HbckReport::getCheckingStartTimestamp)
long startTimestamp = 0; .map(start -> ZonedDateTime.ofInstant(start, ZoneId.systemDefault()))
long endTimestamp = 0; .map(zdt -> zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME))
if (hbckChore != null) { .orElse(null);
inconsistentRegions = hbckChore.getInconsistentRegions(); String hbckReportEndTime = Optional.ofNullable(hbckReport)
orphanRegionsOnRS = hbckChore.getOrphanRegionsOnRS(); .map(HbckReport::getCheckingEndTimestamp)
orphanRegionsOnFS = hbckChore.getOrphanRegionsOnFS(); .map(start -> ZonedDateTime.ofInstant(start, ZoneId.systemDefault()))
startTimestamp = hbckChore.getCheckingStartTimestamp(); .map(zdt -> zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME))
endTimestamp = hbckChore.getCheckingEndTimestamp(); .orElse(null);
}
ZonedDateTime zdt = ZonedDateTime.ofInstant(Instant.ofEpochMilli(startTimestamp),
ZoneId.systemDefault());
String iso8601start = startTimestamp == 0? "-1": zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
zdt = ZonedDateTime.ofInstant(Instant.ofEpochMilli(endTimestamp),
ZoneId.systemDefault());
String iso8601end = startTimestamp == 0? "-1": zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
CatalogJanitor cj = master.getCatalogJanitor(); CatalogJanitor cj = master.getCatalogJanitor();
Report report = cj == null? null: cj.getLastReport(); CatalogJanitorReport cjReport = cj == null? null: cj.getLastReport();
final ServerManager serverManager = master.getServerManager(); final ServerManager serverManager = master.getServerManager();
%> %>
<jsp:include page="header.jsp"> <jsp:include page="header.jsp">
@ -109,20 +104,22 @@
<div class="page-header"> <div class="page-header">
<h1>HBCK Chore Report</h1> <h1>HBCK Chore Report</h1>
<p> <p>
<% if (hbckChore.isDisabled()) { %> <% if (hbckChore == null) { %>
<span>HBCK chore has not yet initialized. Try again later.</span>
<% } else if (hbckChore.isDisabled()) { %>
<span>HBCK chore is currently disabled. Set hbase.master.hbck.chore.interval > 0 in the config & do a rolling-restart to enable it.</span> <span>HBCK chore is currently disabled. Set hbase.master.hbck.chore.interval > 0 in the config & do a rolling-restart to enable it.</span>
<% } else if (startTimestamp == 0 && endTimestamp == 0){ %> <% } else if (hbckReport == null) { %>
<span>No report created.</span> <span>No Report created.</span>
<% } else if (startTimestamp > 0 && endTimestamp == 0){ %> <% } else if (hbckReportStartTime != null && hbckReportEndTime == null) { %>
<span>Checking started at <%= iso8601start %>. Please wait for checking to generate a new sub-report.</span> <span>Checking started at <%= hbckReportStartTime %>. Please wait for checking to generate a new sub-report.</span>
<% } else { %> <% } else { %>
<span>Checking started at <%= iso8601start %> and generated report at <%= iso8601end %>.</span> <span>Checking started at <%= hbckReportStartTime %> and generated catalogJanitorReport at <%= hbckReportEndTime %>.</span>
<% } %> <% } %>
</p> </p>
</div> </div>
</div> </div>
<% if (inconsistentRegions != null && inconsistentRegions.size() > 0) { %> <% if (hbckReport != null && hbckReport.getInconsistentRegions().size() > 0) { %>
<div class="row"> <div class="row">
<div class="page-header"> <div class="page-header">
<h2>Inconsistent Regions</h2> <h2>Inconsistent Regions</h2>
@ -145,7 +142,7 @@
<th>Location in META</th> <th>Location in META</th>
<th>Reported Online RegionServers</th> <th>Reported Online RegionServers</th>
</tr> </tr>
<% for (Map.Entry<String, Pair<ServerName, List<ServerName>>> entry : inconsistentRegions.entrySet()) {%> <% for (Map.Entry<String, Pair<ServerName, List<ServerName>>> entry : hbckReport.getInconsistentRegions().entrySet()) { %>
<tr> <tr>
<td><%= entry.getKey() %></td> <td><%= entry.getKey() %></td>
<td><%= formatServerName(master, serverManager, entry.getValue().getFirst()) %></td> <td><%= formatServerName(master, serverManager, entry.getValue().getFirst()) %></td>
@ -153,11 +150,11 @@
collect(Collectors.joining(", ")) %></td> collect(Collectors.joining(", ")) %></td>
</tr> </tr>
<% } %> <% } %>
<p><%= inconsistentRegions.size() %> region(s) in set.</p> <p><%= hbckReport.getInconsistentRegions().size() %> region(s) in set.</p>
</table> </table>
<% } %> <% } %>
<% if (orphanRegionsOnRS != null && orphanRegionsOnRS.size() > 0) { %> <% if (hbckReport != null && hbckReport.getOrphanRegionsOnRS().size() > 0) { %>
<div class="row"> <div class="row">
<div class="page-header"> <div class="page-header">
<h2>Orphan Regions on RegionServer</h2> <h2>Orphan Regions on RegionServer</h2>
@ -169,17 +166,17 @@
<th>Region Name</th> <th>Region Name</th>
<th>Reported Online RegionServer</th> <th>Reported Online RegionServer</th>
</tr> </tr>
<% for (Map.Entry<String, ServerName> entry : orphanRegionsOnRS.entrySet()) { %> <% for (Map.Entry<String, ServerName> entry : hbckReport.getOrphanRegionsOnRS().entrySet()) { %>
<tr> <tr>
<td><%= entry.getKey() %></td> <td><%= entry.getKey() %></td>
<td><%= formatServerName(master, serverManager, entry.getValue()) %></td> <td><%= formatServerName(master, serverManager, entry.getValue()) %></td>
</tr> </tr>
<% } %> <% } %>
<p><%= orphanRegionsOnRS.size() %> region(s) in set.</p> <p><%= hbckReport.getOrphanRegionsOnRS().size() %> region(s) in set.</p>
</table> </table>
<% } %> <% } %>
<% if (orphanRegionsOnFS != null && orphanRegionsOnFS.size() > 0) { %> <% if (hbckReport != null && hbckReport.getOrphanRegionsOnFS().size() > 0) { %>
<div class="row"> <div class="row">
<div class="page-header"> <div class="page-header">
<h2>Orphan Regions on FileSystem</h2> <h2>Orphan Regions on FileSystem</h2>
@ -200,41 +197,41 @@
<th>Region Encoded Name</th> <th>Region Encoded Name</th>
<th>FileSystem Path</th> <th>FileSystem Path</th>
</tr> </tr>
<% for (Map.Entry<String, Path> entry : orphanRegionsOnFS.entrySet()) { %> <% for (Map.Entry<String, Path> entry : hbckReport.getOrphanRegionsOnFS().entrySet()) { %>
<tr> <tr>
<td><%= entry.getKey() %></td> <td><%= entry.getKey() %></td>
<td><%= entry.getValue() %></td> <td><%= entry.getValue() %></td>
</tr> </tr>
<% } %> <% } %>
<p><%= orphanRegionsOnFS.size() %> region(s) in set.</p> <p><%= hbckReport.getOrphanRegionsOnFS().size() %> region(s) in set.</p>
</table> </table>
<% } %> <% } %>
<% <%
zdt = ZonedDateTime.ofInstant(Instant.ofEpochMilli(System.currentTimeMillis()), Instant nowInstant = Instant.ofEpochMilli(EnvironmentEdgeManager.currentTime());
ZoneId.systemDefault()); ZonedDateTime nowZdt = ZonedDateTime.ofInstant(nowInstant, ZoneId.systemDefault());
String iso8601Now = zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); String iso8601Now = nowZdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME);
String iso8601reportTime = "-1"; String cjReportTime = Optional.ofNullable(cjReport)
if (report != null) { .map(CatalogJanitorReport::getCreateTime)
zdt = ZonedDateTime.ofInstant(Instant.ofEpochMilli(report.getCreateTime()), .map(Instant::ofEpochMilli)
ZoneId.systemDefault()); .map(start -> ZonedDateTime.ofInstant(start, ZoneId.systemDefault()))
iso8601reportTime = zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); .map(zdt -> zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME))
} .orElse(null);
%> %>
<div class="row inner_header"> <div class="row inner_header">
<div class="page-header"> <div class="page-header">
<h1>CatalogJanitor <em>hbase:meta</em> Consistency Issues</h1> <h1>CatalogJanitor <em>hbase:meta</em> Consistency Issues</h1>
<p> <p>
<% if (report != null) { %> <% if (cjReport != null) { %>
<span>Report created: <%= iso8601reportTime %> (now=<%= iso8601Now %>).</span></p> <span>Report created: <%= cjReportTime %> (now=<%= iso8601Now %>).</span></p>
<% } else { %> <% } else { %>
<span>No report created.</span> <span>No catalogJanitorReport created.</span>
<% } %> <% } %>
</div> </div>
</div> </div>
<% if (report != null && !report.isEmpty()) { %> <% if (cjReport != null && !cjReport.isEmpty()) { %>
<% if (!report.getHoles().isEmpty()) { %> <% if (!cjReport.getHoles().isEmpty()) { %>
<div class="row inner_header"> <div class="row inner_header">
<div class="page-header"> <div class="page-header">
<h2>Holes</h2> <h2>Holes</h2>
@ -245,17 +242,17 @@
<th>RegionInfo</th> <th>RegionInfo</th>
<th>RegionInfo</th> <th>RegionInfo</th>
</tr> </tr>
<% for (Pair<RegionInfo, RegionInfo> p : report.getHoles()) { %> <% for (Pair<RegionInfo, RegionInfo> p : cjReport.getHoles()) { %>
<tr> <tr>
<td><span title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td> <td><span title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td>
<td><span title="<%= p.getSecond() %>"><%= p.getSecond().getRegionNameAsString() %></span></td> <td><span title="<%= p.getSecond() %>"><%= p.getSecond().getRegionNameAsString() %></span></td>
</tr> </tr>
<% } %> <% } %>
<p><%= report.getHoles().size() %> hole(s).</p> <p><%= cjReport.getHoles().size() %> hole(s).</p>
</table> </table>
<% } %> <% } %>
<% if (!report.getOverlaps().isEmpty()) { %> <% if (!cjReport.getOverlaps().isEmpty()) { %>
<div class="row inner_header"> <div class="row inner_header">
<div class="page-header"> <div class="page-header">
<h2>Overlaps</h2> <h2>Overlaps</h2>
@ -272,14 +269,14 @@
<th>RegionInfo</th> <th>RegionInfo</th>
<th>Other RegionInfo</th> <th>Other RegionInfo</th>
</tr> </tr>
<% for (Pair<RegionInfo, RegionInfo> p : report.getOverlaps()) { %> <% for (Pair<RegionInfo, RegionInfo> p : cjReport.getOverlaps()) { %>
<tr> <tr>
<% if (report.getMergedRegions().containsKey(p.getFirst())) { %> <% if (cjReport.getMergedRegions().containsKey(p.getFirst())) { %>
<td><span style="color:blue;" title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td> <td><span style="color:blue;" title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td>
<% } else { %> <% } else { %>
<td><span title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td> <td><span title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td>
<% } %> <% } %>
<% if (report.getMergedRegions().containsKey(p.getSecond())) { %> <% if (cjReport.getMergedRegions().containsKey(p.getSecond())) { %>
<td><span style="color:blue;" title="<%= p.getSecond() %>"><%= p.getSecond().getRegionNameAsString() %></span></td> <td><span style="color:blue;" title="<%= p.getSecond() %>"><%= p.getSecond().getRegionNameAsString() %></span></td>
<% } else { %> <% } else { %>
<td><span title="<%= p.getSecond() %>"><%= p.getSecond().getRegionNameAsString() %></span></td> <td><span title="<%= p.getSecond() %>"><%= p.getSecond().getRegionNameAsString() %></span></td>
@ -287,10 +284,10 @@
</tr> </tr>
<% } %> <% } %>
<p><%= report.getOverlaps().size() %> overlap(s).</p> <p><%= cjReport.getOverlaps().size() %> overlap(s).</p>
</table> </table>
<% } %> <% } %>
<% if (!report.getUnknownServers().isEmpty()) { %> <% if (!cjReport.getUnknownServers().isEmpty()) { %>
<div class="row inner_header"> <div class="row inner_header">
<div class="page-header"> <div class="page-header">
<h2>Unknown Servers</h2> <h2>Unknown Servers</h2>
@ -316,17 +313,17 @@
<th>RegionInfo</th> <th>RegionInfo</th>
<th>ServerName</th> <th>ServerName</th>
</tr> </tr>
<% for (Pair<RegionInfo, ServerName> p: report.getUnknownServers()) { %> <% for (Pair<RegionInfo, ServerName> p: cjReport.getUnknownServers()) { %>
<tr> <tr>
<td><span title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td> <td><span title="<%= p.getFirst() %>"><%= p.getFirst().getRegionNameAsString() %></span></td>
<td><%= p.getSecond() %></td> <td><%= p.getSecond() %></td>
</tr> </tr>
<% } %> <% } %>
<p><%= report.getUnknownServers().size() %> unknown servers(s).</p> <p><%= cjReport.getUnknownServers().size() %> unknown servers(s).</p>
</table> </table>
<% } %> <% } %>
<% if (!report.getEmptyRegionInfo().isEmpty()) { %> <% if (!cjReport.getEmptyRegionInfo().isEmpty()) { %>
<div class="row inner_header"> <div class="row inner_header">
<div class="page-header"> <div class="page-header">
<h2>Empty <em>info:regioninfo</em></h2> <h2>Empty <em>info:regioninfo</em></h2>
@ -336,13 +333,13 @@
<tr> <tr>
<th>Row</th> <th>Row</th>
</tr> </tr>
<% for (byte [] row: report.getEmptyRegionInfo()) { %> <% for (byte [] row: cjReport.getEmptyRegionInfo()) { %>
<tr> <tr>
<td><%= Bytes.toStringBinary(row) %></td> <td><%= Bytes.toStringBinary(row) %></td>
</tr> </tr>
<% } %> <% } %>
<p><%= report.getEmptyRegionInfo().size() %> emptyRegionInfo(s).</p> <p><%= cjReport.getEmptyRegionInfo().size() %> emptyRegionInfo(s).</p>
</table> </table>
<% } %> <% } %>
<% } %> <% } %>

View File

@ -19,15 +19,18 @@ package org.apache.hadoop.hbase.client;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import java.time.Instant;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.CoprocessorEnvironment;
@ -42,6 +45,8 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.hbck.HbckChore;
import org.apache.hadoop.hbase.master.hbck.HbckReport;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface; import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.Procedure;
@ -302,15 +307,14 @@ public class TestHbck {
@Test @Test
public void testRunHbckChore() throws Exception { public void testRunHbckChore() throws Exception {
HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster(); HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
long endTimestamp = master.getHbckChore().getCheckingEndTimestamp(); HbckChore hbckChore = master.getHbckChore();
Instant endTimestamp = Optional.ofNullable(hbckChore.getLastReport())
.map(HbckReport::getCheckingEndTimestamp).orElse(Instant.EPOCH);
Hbck hbck = getHbck(); Hbck hbck = getHbck();
boolean ran = false; TEST_UTIL.waitFor(TimeUnit.MINUTES.toMillis(5), hbck::runHbckChore);
while (!ran) { HbckReport report = hbckChore.getLastReport();
ran = hbck.runHbckChore(); assertNotNull(report);
if (ran) { assertTrue(report.getCheckingEndTimestamp().isAfter(endTimestamp));
assertTrue(master.getHbckChore().getCheckingEndTimestamp() > endTimestamp);
}
}
} }
public static class FailingSplitAfterMetaUpdatedMasterObserver public static class FailingSplitAfterMetaUpdatedMasterObserver

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner; import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner; import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
import org.apache.hadoop.hbase.master.hbck.HbckChore;
import org.apache.hadoop.hbase.master.janitor.CatalogJanitor; import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;

View File

@ -19,11 +19,16 @@ package org.apache.hadoop.hbase.master.assignment;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.time.Instant;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
@ -33,8 +38,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.HbckChore;
import org.apache.hadoop.hbase.master.TableStateManager; import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.master.hbck.HbckChore;
import org.apache.hadoop.hbase.master.hbck.HbckReport;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -75,7 +81,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
hbckChore.choreForTesting(); hbckChore.choreForTesting();
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions = Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
hbckChore.getInconsistentRegions(); hbckChore.getLastReport().getInconsistentRegions();
// Test for case1: Master thought this region opened, but no regionserver reported it. // Test for case1: Master thought this region opened, but no regionserver reported it.
assertTrue(inconsistentRegions.containsKey(metaRegionName)); assertTrue(inconsistentRegions.containsKey(metaRegionName));
@ -88,7 +94,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
// Reported right region location. Then not in problematic regions. // Reported right region location. Then not in problematic regions.
am.reportOnlineRegions(locationInMeta, Collections.singleton(metaRegionNameAsBytes)); am.reportOnlineRegions(locationInMeta, Collections.singleton(metaRegionNameAsBytes));
hbckChore.choreForTesting(); hbckChore.choreForTesting();
inconsistentRegions = hbckChore.getInconsistentRegions(); inconsistentRegions = hbckChore.getLastReport().getInconsistentRegions();
assertFalse(inconsistentRegions.containsKey(metaRegionName)); assertFalse(inconsistentRegions.containsKey(metaRegionName));
} }
@ -107,7 +113,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
// Test for case1: Master thought this region opened, but no regionserver reported it. // Test for case1: Master thought this region opened, but no regionserver reported it.
hbckChore.choreForTesting(); hbckChore.choreForTesting();
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions = Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
hbckChore.getInconsistentRegions(); hbckChore.getLastReport().getInconsistentRegions();
assertTrue(inconsistentRegions.containsKey(regionName)); assertTrue(inconsistentRegions.containsKey(regionName));
Pair<ServerName, List<ServerName>> pair = inconsistentRegions.get(regionName); Pair<ServerName, List<ServerName>> pair = inconsistentRegions.get(regionName);
ServerName locationInMeta = pair.getFirst(); ServerName locationInMeta = pair.getFirst();
@ -122,7 +128,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
serverNames.stream().filter(s -> !s.equals(tempLocationInMeta)).findFirst().get(); serverNames.stream().filter(s -> !s.equals(tempLocationInMeta)).findFirst().get();
am.reportOnlineRegions(anotherServer, Collections.singleton(hri.getRegionName())); am.reportOnlineRegions(anotherServer, Collections.singleton(hri.getRegionName()));
hbckChore.choreForTesting(); hbckChore.choreForTesting();
inconsistentRegions = hbckChore.getInconsistentRegions(); inconsistentRegions = hbckChore.getLastReport().getInconsistentRegions();
assertTrue(inconsistentRegions.containsKey(regionName)); assertTrue(inconsistentRegions.containsKey(regionName));
pair = inconsistentRegions.get(regionName); pair = inconsistentRegions.get(regionName);
locationInMeta = pair.getFirst(); locationInMeta = pair.getFirst();
@ -134,7 +140,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
// Test for case3: More than one regionservers reported opened this region. // Test for case3: More than one regionservers reported opened this region.
am.reportOnlineRegions(locationInMeta, Collections.singleton(hri.getRegionName())); am.reportOnlineRegions(locationInMeta, Collections.singleton(hri.getRegionName()));
hbckChore.choreForTesting(); hbckChore.choreForTesting();
inconsistentRegions = hbckChore.getInconsistentRegions(); inconsistentRegions = hbckChore.getLastReport().getInconsistentRegions();
assertTrue(inconsistentRegions.containsKey(regionName)); assertTrue(inconsistentRegions.containsKey(regionName));
pair = inconsistentRegions.get(regionName); pair = inconsistentRegions.get(regionName);
locationInMeta = pair.getFirst(); locationInMeta = pair.getFirst();
@ -146,7 +152,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
// Reported right region location, then not in inconsistent regions. // Reported right region location, then not in inconsistent regions.
am.reportOnlineRegions(anotherServer, Collections.EMPTY_SET); am.reportOnlineRegions(anotherServer, Collections.EMPTY_SET);
hbckChore.choreForTesting(); hbckChore.choreForTesting();
inconsistentRegions = hbckChore.getInconsistentRegions(); inconsistentRegions = hbckChore.getLastReport().getInconsistentRegions();
assertFalse(inconsistentRegions.containsKey(regionName)); assertFalse(inconsistentRegions.containsKey(regionName));
} }
@ -164,7 +170,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
hbckChore.choreForTesting(); hbckChore.choreForTesting();
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions = Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
hbckChore.getInconsistentRegions(); hbckChore.getLastReport().getInconsistentRegions();
assertTrue(inconsistentRegions.containsKey(regionName)); assertTrue(inconsistentRegions.containsKey(regionName));
Pair<ServerName, List<ServerName>> pair = inconsistentRegions.get(regionName); Pair<ServerName, List<ServerName>> pair = inconsistentRegions.get(regionName);
ServerName locationInMeta = pair.getFirst(); ServerName locationInMeta = pair.getFirst();
@ -177,7 +183,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
Mockito.when(tableStateManager.isTableState(tableName, TableState.State.DISABLED)) Mockito.when(tableStateManager.isTableState(tableName, TableState.State.DISABLED))
.thenReturn(true); .thenReturn(true);
hbckChore.choreForTesting(); hbckChore.choreForTesting();
inconsistentRegions = hbckChore.getInconsistentRegions(); inconsistentRegions = hbckChore.getLastReport().getInconsistentRegions();
assertFalse(inconsistentRegions.containsKey(regionName)); assertFalse(inconsistentRegions.containsKey(regionName));
} }
@ -196,7 +202,7 @@ public class TestHbckChore extends TestAssignmentManagerBase {
hbckChore.choreForTesting(); hbckChore.choreForTesting();
Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions = Map<String, Pair<ServerName, List<ServerName>>> inconsistentRegions =
hbckChore.getInconsistentRegions(); hbckChore.getLastReport().getInconsistentRegions();
assertFalse(inconsistentRegions.containsKey(regionName)); assertFalse(inconsistentRegions.containsKey(regionName));
} }
@ -207,33 +213,35 @@ public class TestHbckChore extends TestAssignmentManagerBase {
Configuration conf = util.getConfiguration(); Configuration conf = util.getConfiguration();
hbckChore.choreForTesting(); hbckChore.choreForTesting();
assertEquals(0, hbckChore.getOrphanRegionsOnFS().size()); assertEquals(0, hbckChore.getLastReport().getOrphanRegionsOnFS().size());
HRegion.createRegionDir(conf, regionInfo, CommonFSUtils.getRootDir(conf)); HRegion.createRegionDir(conf, regionInfo, CommonFSUtils.getRootDir(conf));
hbckChore.choreForTesting(); hbckChore.choreForTesting();
assertEquals(1, hbckChore.getOrphanRegionsOnFS().size()); assertEquals(1, hbckChore.getLastReport().getOrphanRegionsOnFS().size());
assertTrue(hbckChore.getOrphanRegionsOnFS().containsKey(regionInfo.getEncodedName())); assertTrue(
hbckChore.getLastReport().getOrphanRegionsOnFS().containsKey(regionInfo.getEncodedName()));
FSUtils.deleteRegionDir(conf, new HRegionInfo(regionInfo)); FSUtils.deleteRegionDir(conf, new HRegionInfo(regionInfo));
hbckChore.choreForTesting(); hbckChore.choreForTesting();
assertEquals(0, hbckChore.getOrphanRegionsOnFS().size()); assertEquals(0, hbckChore.getLastReport().getOrphanRegionsOnFS().size());
} }
@Test @Test
public void testChoreDisable() { public void testChoreDisable() {
// The way to disable to chore is to set hbase.master.hbck.chore.interval <= 0 // The way to disable to chore is to set hbase.master.hbck.chore.interval <= 0
// When the interval is > 0, the chore should run. // When the interval is > 0, the chore should run.
long lastRunTime = hbckChore.getCheckingEndTimestamp(); Instant lastRunTime = Optional.ofNullable(hbckChore.getLastReport())
.map(HbckReport::getCheckingEndTimestamp).orElse(null);
hbckChore.choreForTesting(); hbckChore.choreForTesting();
boolean ran = lastRunTime != hbckChore.getCheckingEndTimestamp(); Instant thisRunTime = Optional.ofNullable(hbckChore.getLastReport())
assertTrue(ran); .map(HbckReport::getCheckingEndTimestamp).orElse(null);
assertNotNull(thisRunTime);
assertNotEquals(lastRunTime, thisRunTime);
// When the interval <= 0, the chore shouldn't run // When the interval <= 0, the chore shouldn't run
master.getConfiguration().setInt("hbase.master.hbck.chore.interval", 0); master.getConfiguration().setInt("hbase.master.hbck.chore.interval", 0);
HbckChore hbckChoreWithChangedConf = new HbckChore(master); HbckChore hbckChoreWithChangedConf = new HbckChore(master);
lastRunTime = hbckChoreWithChangedConf.getCheckingEndTimestamp();
hbckChoreWithChangedConf.choreForTesting(); hbckChoreWithChangedConf.choreForTesting();
ran = lastRunTime != hbckChoreWithChangedConf.getCheckingEndTimestamp(); assertNull(hbckChoreWithChangedConf.getLastReport());
assertFalse(ran);
} }
} }

View File

@ -313,7 +313,7 @@ public class TestCatalogJanitor {
final Map<HRegionInfo, Result> mergedRegions = new TreeMap<>(); final Map<HRegionInfo, Result> mergedRegions = new TreeMap<>();
CatalogJanitor spy = spy(this.janitor); CatalogJanitor spy = spy(this.janitor);
Report report = new Report(); CatalogJanitorReport report = new CatalogJanitorReport();
report.count = 10; report.count = 10;
report.mergedRegions.putAll(mergedRegions); report.mergedRegions.putAll(mergedRegions);
report.splitParents.putAll(splitParents); report.splitParents.putAll(splitParents);

View File

@ -103,7 +103,7 @@ public class TestCatalogJanitorCluster {
RegionStateStore regionStateStore = RegionStateStore regionStateStore =
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore(); TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
janitor.scan(); janitor.scan();
Report report = janitor.getLastReport(); CatalogJanitorReport report = janitor.getLastReport();
// Assert no problems. // Assert no problems.
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
// Now remove first region in table t2 to see if catalogjanitor scan notices. // Now remove first region in table t2 to see if catalogjanitor scan notices.
@ -233,7 +233,7 @@ public class TestCatalogJanitorCluster {
public void testHoles() throws IOException { public void testHoles() throws IOException {
CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor(); CatalogJanitor janitor = TEST_UTIL.getHBaseCluster().getMaster().getCatalogJanitor();
Report report = janitor.getLastReport(); CatalogJanitorReport report = janitor.getLastReport();
// Assert no problems. // Assert no problems.
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
// Verify start and end region holes // Verify start and end region holes
@ -248,7 +248,7 @@ public class TestCatalogJanitorCluster {
private void fixHoles(CatalogJanitor janitor) throws IOException { private void fixHoles(CatalogJanitor janitor) throws IOException {
MetaFixer metaFixer = new MetaFixer(TEST_UTIL.getHBaseCluster().getMaster()); MetaFixer metaFixer = new MetaFixer(TEST_UTIL.getHBaseCluster().getMaster());
janitor.scan(); janitor.scan();
Report report = janitor.getLastReport(); CatalogJanitorReport report = janitor.getLastReport();
// Verify total number of holes, 2 in t1 and t2 each and one in t3 // Verify total number of holes, 2 in t1 and t2 each and one in t3
assertEquals("Number of holes are not matching", 5, report.getHoles().size()); assertEquals("Number of holes are not matching", 5, report.getHoles().size());
metaFixer.fix(); metaFixer.fix();
@ -303,7 +303,7 @@ public class TestCatalogJanitorCluster {
private LinkedList<Pair<RegionInfo, RegionInfo>> getHoles(CatalogJanitor janitor, private LinkedList<Pair<RegionInfo, RegionInfo>> getHoles(CatalogJanitor janitor,
TableName tableName) throws IOException { TableName tableName) throws IOException {
janitor.scan(); janitor.scan();
Report lastReport = janitor.getLastReport(); CatalogJanitorReport lastReport = janitor.getLastReport();
assertFalse(lastReport.isEmpty()); assertFalse(lastReport.isEmpty());
LinkedList<Pair<RegionInfo, RegionInfo>> holes = new LinkedList<>(); LinkedList<Pair<RegionInfo, RegionInfo>> holes = new LinkedList<>();
for (Pair<RegionInfo, RegionInfo> hole : lastReport.getHoles()) { for (Pair<RegionInfo, RegionInfo> hole : lastReport.getHoles()) {

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.master.janitor; package org.apache.hadoop.hbase.master.janitor;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
@ -39,12 +40,13 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.HbckChore;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.GCMultipleMergedRegionsProcedure; import org.apache.hadoop.hbase.master.assignment.GCMultipleMergedRegionsProcedure;
import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure; import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.hbck.HbckChore;
import org.apache.hadoop.hbase.master.hbck.HbckReport;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@ -95,7 +97,7 @@ public class TestMetaFixer {
MasterServices services = TEST_UTIL.getHBaseCluster().getMaster(); MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
int initialSize = services.getAssignmentManager().getRegionStates().getRegionStates().size(); int initialSize = services.getAssignmentManager().getRegionStates().getRegionStates().size();
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
Report report = services.getCatalogJanitor().getLastReport(); CatalogJanitorReport report = services.getCatalogJanitor().getLastReport();
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
int originalCount = ris.size(); int originalCount = ris.size();
// Remove first, last and middle region. See if hole gets plugged. Table has 26 * replicaCount // Remove first, last and middle region. See if hole gets plugged. Table has 26 * replicaCount
@ -153,7 +155,7 @@ public class TestMetaFixer {
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
deleteRegion(services, ris.get(0)); deleteRegion(services, ris.get(0));
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
Report report = services.getCatalogJanitor().getLastReport(); CatalogJanitorReport report = services.getCatalogJanitor().getLastReport();
ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn); ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn);
assertTrue(ris.isEmpty()); assertTrue(ris.isEmpty());
MetaFixer fixer = new MetaFixer(services); MetaFixer fixer = new MetaFixer(services);
@ -184,7 +186,7 @@ public class TestMetaFixer {
assertTrue(ris.size() > 5); assertTrue(ris.size() > 5);
HMaster services = TEST_UTIL.getHBaseCluster().getMaster(); HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
Report report = services.getCatalogJanitor().getLastReport(); CatalogJanitorReport report = services.getCatalogJanitor().getLastReport();
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
// Make a simple overlap spanning second and third region. // Make a simple overlap spanning second and third region.
makeOverlap(services, ris.get(1), ris.get(3)); makeOverlap(services, ris.get(1), ris.get(3));
@ -201,7 +203,7 @@ public class TestMetaFixer {
CatalogJanitor cj = services.getCatalogJanitor(); CatalogJanitor cj = services.getCatalogJanitor();
cj.scan(); cj.scan();
Report report = cj.getLastReport(); CatalogJanitorReport report = cj.getLastReport();
assertEquals(6, report.getOverlaps().size()); assertEquals(6, report.getOverlaps().size());
assertEquals(1, MetaFixer.calculateMerges(10, report.getOverlaps()).size()); assertEquals(1, MetaFixer.calculateMerges(10, report.getOverlaps()).size());
MetaFixer fixer = new MetaFixer(services); MetaFixer fixer = new MetaFixer(services);
@ -237,11 +239,13 @@ public class TestMetaFixer {
// No orphan regions on FS // No orphan regions on FS
hbckChore.choreForTesting(); hbckChore.choreForTesting();
assertEquals(0, hbckChore.getOrphanRegionsOnFS().size()); HbckReport hbckReport = hbckChore.getLastReport();
assertNotNull(hbckReport);
assertEquals(0, hbckReport.getOrphanRegionsOnFS().size());
// No holes reported. // No holes reported.
cj.scan(); cj.scan();
final Report postReport = cj.getLastReport(); final CatalogJanitorReport postReport = cj.getLastReport();
assertTrue(postReport.isEmpty()); assertTrue(postReport.isEmpty());
} }
@ -255,7 +259,7 @@ public class TestMetaFixer {
HMaster services = TEST_UTIL.getHBaseCluster().getMaster(); HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
Report report = services.getCatalogJanitor().getLastReport(); CatalogJanitorReport report = services.getCatalogJanitor().getLastReport();
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
// Make a simple overlap for t1 // Make a simple overlap for t1
@ -288,7 +292,7 @@ public class TestMetaFixer {
HMaster services = TEST_UTIL.getHBaseCluster().getMaster(); HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
CatalogJanitor cj = services.getCatalogJanitor(); CatalogJanitor cj = services.getCatalogJanitor();
cj.scan(); cj.scan();
Report report = cj.getLastReport(); CatalogJanitorReport report = cj.getLastReport();
assertEquals(6, report.getOverlaps().size()); assertEquals(6, report.getOverlaps().size());
assertEquals(2, MetaFixer.calculateMerges(5, report.getOverlaps()).size()); assertEquals(2, MetaFixer.calculateMerges(5, report.getOverlaps()).size());
@ -311,7 +315,7 @@ public class TestMetaFixer {
HBaseTestingUtility.await(200, () -> { HBaseTestingUtility.await(200, () -> {
try { try {
cj.scan(); cj.scan();
final Report postReport = cj.getLastReport(); final CatalogJanitorReport postReport = cj.getLastReport();
RegionStates regionStates = am.getRegionStates(); RegionStates regionStates = am.getRegionStates();
// Make sure that two merged regions are opened and GCs are done. // Make sure that two merged regions are opened and GCs are done.
@ -354,7 +358,7 @@ public class TestMetaFixer {
// No holes reported. // No holes reported.
cj.scan(); cj.scan();
final Report postReport = cj.getLastReport(); final CatalogJanitorReport postReport = cj.getLastReport();
assertTrue(postReport.isEmpty()); assertTrue(postReport.isEmpty());
} finally { } finally {
@ -379,7 +383,7 @@ public class TestMetaFixer {
HMaster services = TEST_UTIL.getHBaseCluster().getMaster(); HMaster services = TEST_UTIL.getHBaseCluster().getMaster();
CatalogJanitor cj = services.getCatalogJanitor(); CatalogJanitor cj = services.getCatalogJanitor();
cj.scan(); cj.scan();
Report report = cj.getLastReport(); CatalogJanitorReport report = cj.getLastReport();
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
RegionInfo overlapRegion = makeOverlap(services, ris.get(1), ris.get(2)); RegionInfo overlapRegion = makeOverlap(services, ris.get(1), ris.get(2));
@ -440,7 +444,7 @@ public class TestMetaFixer {
assertTrue(ris.size() > 5); assertTrue(ris.size() > 5);
MasterServices services = TEST_UTIL.getHBaseCluster().getMaster(); MasterServices services = TEST_UTIL.getHBaseCluster().getMaster();
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
Report report = services.getCatalogJanitor().getLastReport(); CatalogJanitorReport report = services.getCatalogJanitor().getLastReport();
assertTrue(report.isEmpty()); assertTrue(report.isEmpty());
// Make a simple overlap spanning second and third region. // Make a simple overlap spanning second and third region.
makeOverlap(services, ris.get(1), ris.get(5)); makeOverlap(services, ris.get(1), ris.get(5));
@ -464,7 +468,7 @@ public class TestMetaFixer {
HBaseTestingUtility.await(10, () -> { HBaseTestingUtility.await(10, () -> {
try { try {
services.getCatalogJanitor().scan(); services.getCatalogJanitor().scan();
final Report postReport = services.getCatalogJanitor().getLastReport(); final CatalogJanitorReport postReport = services.getCatalogJanitor().getLastReport();
return postReport.isEmpty(); return postReport.isEmpty();
} catch (Exception e) { } catch (Exception e) {
throw new RuntimeException(e); throw new RuntimeException(e);