HBASE-25509 ChoreService.cancelChore will not call ScheduledChore.cle… (#2890)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Duo Zhang 2021-01-20 16:10:36 +08:00 committed by GitHub
parent 871eb09b3d
commit a37e727990
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 704 additions and 785 deletions

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase;
import com.google.errorprone.annotations.RestrictedApi;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.LinkedHashMap;
@ -26,8 +27,6 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.ScheduledChore.ChoreServicer;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -52,7 +51,7 @@ import org.slf4j.LoggerFactory;
* Calling this method ensures that all scheduled chores are cancelled and cleaned up properly.
*/
@InterfaceAudience.Public
public class ChoreService implements ChoreServicer {
public class ChoreService {
private static final Logger LOG = LoggerFactory.getLogger(ChoreService.class);
/**
@ -141,28 +140,39 @@ public class ChoreService implements ChoreServicer {
* @return true when the chore was successfully scheduled. false when the scheduling failed
* (typically occurs when a chore is scheduled during shutdown of service)
*/
public synchronized boolean scheduleChore(ScheduledChore chore) {
public boolean scheduleChore(ScheduledChore chore) {
if (chore == null) {
return false;
}
try {
if (chore.getPeriod() <= 0) {
LOG.info("Chore {} is disabled because its period is not positive.", chore);
return false;
// always lock chore first to prevent dead lock
synchronized (chore) {
synchronized (this) {
try {
// Chores should only ever be scheduled with a single ChoreService. If the choreService
// is changing, cancel any existing schedules of this chore.
if (chore.getChoreService() == this) {
LOG.warn("Chore {} has already been scheduled with us", chore);
return false;
}
if (chore.getPeriod() <= 0) {
LOG.info("Chore {} is disabled because its period is not positive.", chore);
return false;
}
LOG.info("Chore {} is enabled.", chore);
if (chore.getChoreService() != null) {
LOG.info("Cancel chore {} from its previous service", chore);
chore.getChoreService().cancelChore(chore);
}
chore.setChoreService(this);
ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(chore, chore.getInitialDelay(),
chore.getPeriod(), chore.getTimeUnit());
scheduledChores.put(chore, future);
return true;
} catch (Exception e) {
LOG.error("Could not successfully schedule chore: {}", chore.getName(), e);
return false;
}
}
LOG.info("Chore {} is enabled.", chore);
chore.setChoreServicer(this);
ScheduledFuture<?> future =
scheduler.scheduleAtFixedRate(chore, chore.getInitialDelay(), chore.getPeriod(),
chore.getTimeUnit());
scheduledChores.put(chore, future);
return true;
} catch (Exception exception) {
if (LOG.isInfoEnabled()) {
LOG.info("Could not successfully schedule chore: " + chore.getName());
}
return false;
}
}
@ -175,19 +185,35 @@ public class ChoreService implements ChoreServicer {
ScheduledFuture<?> future = scheduledChores.get(chore);
future.cancel(false);
}
scheduleChore(chore);
ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(chore, chore.getInitialDelay(),
chore.getPeriod(), chore.getTimeUnit());
scheduledChores.put(chore, future);
}
@InterfaceAudience.Private
@Override
public synchronized void cancelChore(ScheduledChore chore) {
/**
* Cancel any ongoing schedules that this chore has with the implementer of this interface.
* <p/>
* Call {@link ScheduledChore#cancel()} to cancel a {@link ScheduledChore}, in
* {@link ScheduledChore#cancel()} method we will call this method to remove the
* {@link ScheduledChore} from this {@link ChoreService}.
*/
@RestrictedApi(explanation = "Should only be called in ScheduledChore", link = "",
allowedOnPath = ".*/org/apache/hadoop/hbase/(ScheduledChore|ChoreService).java")
synchronized void cancelChore(ScheduledChore chore) {
cancelChore(chore, true);
}
@InterfaceAudience.Private
@Override
public synchronized void cancelChore(ScheduledChore chore, boolean mayInterruptIfRunning) {
if (chore != null && scheduledChores.containsKey(chore)) {
/**
* Cancel any ongoing schedules that this chore has with the implementer of this interface.
* <p/>
* Call {@link ScheduledChore#cancel(boolean)} to cancel a {@link ScheduledChore}, in
* {@link ScheduledChore#cancel(boolean)} method we will call this method to remove the
* {@link ScheduledChore} from this {@link ChoreService}.
*/
@RestrictedApi(explanation = "Should only be called in ScheduledChore", link = "",
allowedOnPath = ".*/org/apache/hadoop/hbase/(ScheduledChore|ChoreService).java")
synchronized void cancelChore(ScheduledChore chore, boolean mayInterruptIfRunning) {
if (scheduledChores.containsKey(chore)) {
ScheduledFuture<?> future = scheduledChores.get(chore);
future.cancel(mayInterruptIfRunning);
scheduledChores.remove(chore);
@ -201,21 +227,24 @@ public class ChoreService implements ChoreServicer {
}
}
/**
* @return true when the chore is scheduled with the implementer of this interface
*/
@InterfaceAudience.Private
@Override
public synchronized boolean isChoreScheduled(ScheduledChore chore) {
return chore != null && scheduledChores.containsKey(chore)
&& !scheduledChores.get(chore).isDone();
}
@InterfaceAudience.Private
@Override
public synchronized boolean triggerNow(ScheduledChore chore) {
if (chore != null) {
rescheduleChore(chore);
return true;
}
return false;
/**
* This method tries to execute the chore immediately. If the chore is executing at the time of
* this call, the chore will begin another execution as soon as the current execution finishes
*/
@RestrictedApi(explanation = "Should only be called in ScheduledChore", link = "",
allowedOnPath = ".*/org/apache/hadoop/hbase/ScheduledChore.java")
synchronized void triggerNow(ScheduledChore chore) {
assert chore.getChoreService() == this;
rescheduleChore(chore);
}
/**
@ -295,10 +324,20 @@ public class ChoreService implements ChoreServicer {
}
}
@InterfaceAudience.Private
@Override
public synchronized void onChoreMissedStartTime(ScheduledChore chore) {
if (chore == null || !scheduledChores.containsKey(chore)) return;
/**
* A callback that tells the implementer of this interface that one of the scheduled chores is
* missing its start time. The implication of a chore missing its start time is that the service's
* current means of scheduling may not be sufficient to handle the number of ongoing chores (the
* other explanation is that the chore's execution time is greater than its scheduled period). The
* service should try to increase its concurrency when this callback is received.
* @param chore The chore that missed its start time
*/
@RestrictedApi(explanation = "Should only be called in ScheduledChore", link = "",
allowedOnPath = ".*/org/apache/hadoop/hbase/ScheduledChore.java")
synchronized void onChoreMissedStartTime(ScheduledChore chore) {
if (!scheduledChores.containsKey(chore)) {
return;
}
// If the chore has not caused an increase in the size of the core thread pool then request an
// increase. This allows each chore missing its start time to increase the core pool size by
@ -319,13 +358,17 @@ public class ChoreService implements ChoreServicer {
* shutdown the service. Any chores that are scheduled for execution will be cancelled. Any chores
* in the middle of execution will be interrupted and shutdown. This service will be unusable
* after this method has been called (i.e. future scheduling attempts will fail).
* <p/>
* Notice that, this will only clean the chore from this ChoreService but you could still schedule
* the chore with other ChoreService.
*/
public synchronized void shutdown() {
scheduler.shutdownNow();
if (LOG.isInfoEnabled()) {
LOG.info("Chore service for: " + coreThreadPoolPrefix + " had " + scheduledChores.keySet()
+ " on shutdown");
if (isShutdown()) {
return;
}
scheduler.shutdownNow();
LOG.info("Chore service for: {} had {} on shutdown", coreThreadPoolPrefix,
scheduledChores.keySet());
cancelAllChores(true);
scheduledChores.clear();
choresMissingStartTime.clear();

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase;
import com.google.errorprone.annotations.RestrictedApi;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
@ -33,7 +34,7 @@ import org.slf4j.LoggerFactory;
* execute within the defined period. It is bad practice to define a ScheduledChore whose execution
* time exceeds its period since it will try to hog one of the threads in the {@link ChoreService}'s
* thread pool.
* <p>
* <p/>
* Don't subclass ScheduledChore if the task relies on being woken up for something to do, such as
* an entry being added to a queue, etc.
*/
@ -60,7 +61,7 @@ public abstract class ScheduledChore implements Runnable {
* Interface to the ChoreService that this ScheduledChore is scheduled with. null if the chore is
* not scheduled.
*/
private ChoreServicer choreServicer;
private ChoreService choreService;
/**
* Variables that encapsulate the meaningful state information
@ -77,39 +78,6 @@ public abstract class ScheduledChore implements Runnable {
*/
private final Stoppable stopper;
interface ChoreServicer {
/**
* Cancel any ongoing schedules that this chore has with the implementer of this interface.
*/
public void cancelChore(ScheduledChore chore);
public void cancelChore(ScheduledChore chore, boolean mayInterruptIfRunning);
/**
* @return true when the chore is scheduled with the implementer of this interface
*/
public boolean isChoreScheduled(ScheduledChore chore);
/**
* This method tries to execute the chore immediately. If the chore is executing at the time of
* this call, the chore will begin another execution as soon as the current execution finishes
* <p>
* If the chore is not scheduled with a ChoreService, this call will fail.
* @return false when the chore could not be triggered immediately
*/
public boolean triggerNow(ScheduledChore chore);
/**
* A callback that tells the implementer of this interface that one of the scheduled chores is
* missing its start time. The implication of a chore missing its start time is that the
* service's current means of scheduling may not be sufficient to handle the number of ongoing
* chores (the other explanation is that the chore's execution time is greater than its
* scheduled period). The service should try to increase its concurrency when this callback is
* received.
* @param chore The chore that missed its start time
*/
public void onChoreMissedStartTime(ScheduledChore chore);
}
/**
* This constructor is for test only. It allows us to create an object and to call chore() on it.
*/
@ -168,8 +136,8 @@ public abstract class ScheduledChore implements Runnable {
onChoreMissedStartTime();
LOG.info("Chore: {} missed its start time", getName());
} else if (stopper.isStopped() || !isScheduled()) {
cancel(false);
cleanup();
// call shutdown here to cleanup the ScheduledChore.
shutdown(false);
LOG.info("Chore: {} was stopped", getName());
} else {
try {
@ -193,7 +161,6 @@ public abstract class ScheduledChore implements Runnable {
LOG.error("Caught error", t);
if (this.stopper.isStopped()) {
cancel(false);
cleanup();
}
}
}
@ -214,7 +181,9 @@ public abstract class ScheduledChore implements Runnable {
* pool threads
*/
private synchronized void onChoreMissedStartTime() {
if (choreServicer != null) choreServicer.onChoreMissedStartTime(this);
if (choreService != null) {
choreService.onChoreMissedStartTime(this);
}
}
/**
@ -253,20 +222,17 @@ public abstract class ScheduledChore implements Runnable {
* @return false when the Chore is not currently scheduled with a ChoreService
*/
public synchronized boolean triggerNow() {
if (choreServicer != null) {
return choreServicer.triggerNow(this);
} else {
if (choreService == null) {
return false;
}
choreService.triggerNow(this);
return true;
}
synchronized void setChoreServicer(ChoreServicer service) {
// Chores should only ever be scheduled with a single ChoreService. If the choreServicer
// is changing, cancel any existing schedules of this chore.
if (choreServicer != null && choreServicer != service) {
choreServicer.cancelChore(this, false);
}
choreServicer = service;
@RestrictedApi(explanation = "Should only be called in ChoreService", link = "",
allowedOnPath = ".*/org/apache/hadoop/hbase/ChoreService.java")
synchronized void setChoreService(ChoreService service) {
choreService = service;
timeOfThisRun = -1;
}
@ -275,9 +241,10 @@ public abstract class ScheduledChore implements Runnable {
}
public synchronized void cancel(boolean mayInterruptIfRunning) {
if (isScheduled()) choreServicer.cancelChore(this, mayInterruptIfRunning);
choreServicer = null;
if (isScheduled()) {
choreService.cancelChore(this, mayInterruptIfRunning);
}
choreService = null;
}
public String getName() {
@ -310,17 +277,14 @@ public abstract class ScheduledChore implements Runnable {
return initialChoreComplete;
}
@InterfaceAudience.Private
synchronized ChoreServicer getChoreServicer() {
return choreServicer;
synchronized ChoreService getChoreService() {
return choreService;
}
@InterfaceAudience.Private
synchronized long getTimeOfLastRun() {
return timeOfLastRun;
}
@InterfaceAudience.Private
synchronized long getTimeOfThisRun() {
return timeOfThisRun;
}
@ -329,10 +293,12 @@ public abstract class ScheduledChore implements Runnable {
* @return true when this Chore is scheduled with a ChoreService
*/
public synchronized boolean isScheduled() {
return choreServicer != null && choreServicer.isChoreScheduled(this);
return choreService != null && choreService.isChoreScheduled(this);
}
@InterfaceAudience.Private
@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public synchronized void choreForTesting() {
chore();
}
@ -354,7 +320,26 @@ public abstract class ScheduledChore implements Runnable {
/**
* Override to run cleanup tasks when the Chore encounters an error and must stop running
*/
protected synchronized void cleanup() {
protected void cleanup() {
}
/**
* Call {@link #shutdown(boolean)} with {@code true}.
* @see ScheduledChore#shutdown(boolean)
*/
public synchronized void shutdown() {
shutdown(true);
}
/**
* Completely shutdown the ScheduleChore, which means we will call cleanup and you should not
* schedule it again.
* <p/>
* This is another path to cleanup the chore, comparing to stop the stopper instance passed in.
*/
public synchronized void shutdown(boolean mayInterruptIfRunning) {
cancel(mayInterruptIfRunning);
cleanup();
}
/**

View File

@ -55,7 +55,6 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
@ -1500,11 +1499,9 @@ public class HMaster extends HRegionServer implements MasterServices {
try {
snapshotCleanupTracker.setSnapshotCleanupEnabled(on);
if (on) {
if (!getChoreService().isChoreScheduled(this.snapshotCleanerChore)) {
getChoreService().scheduleChore(this.snapshotCleanerChore);
}
getChoreService().scheduleChore(this.snapshotCleanerChore);
} else {
getChoreService().cancelChore(this.snapshotCleanerChore);
this.snapshotCleanerChore.cancel();
}
} catch (KeeperException e) {
LOG.error("Error updating snapshot cleanup mode to {}", on, e);
@ -1528,24 +1525,23 @@ public class HMaster extends HRegionServer implements MasterServices {
}
private void stopChores() {
ChoreService choreService = getChoreService();
if (choreService != null) {
choreService.cancelChore(this.mobFileCleanerChore);
choreService.cancelChore(this.mobFileCompactionChore);
choreService.cancelChore(this.balancerChore);
if (getChoreService() != null) {
shutdownChore(mobFileCleanerChore);
shutdownChore(mobFileCompactionChore);
shutdownChore(balancerChore);
if (regionNormalizerManager != null) {
choreService.cancelChore(regionNormalizerManager.getRegionNormalizerChore());
shutdownChore(regionNormalizerManager.getRegionNormalizerChore());
}
choreService.cancelChore(this.clusterStatusChore);
choreService.cancelChore(this.catalogJanitorChore);
choreService.cancelChore(this.clusterStatusPublisherChore);
choreService.cancelChore(this.snapshotQuotaChore);
choreService.cancelChore(this.logCleaner);
choreService.cancelChore(this.hfileCleaner);
choreService.cancelChore(this.replicationBarrierCleaner);
choreService.cancelChore(this.snapshotCleanerChore);
choreService.cancelChore(this.hbckChore);
choreService.cancelChore(this.regionsRecoveryChore);
shutdownChore(clusterStatusChore);
shutdownChore(catalogJanitorChore);
shutdownChore(clusterStatusPublisherChore);
shutdownChore(snapshotQuotaChore);
shutdownChore(logCleaner);
shutdownChore(hfileCleaner);
shutdownChore(replicationBarrierCleaner);
shutdownChore(snapshotCleanerChore);
shutdownChore(hbckChore);
shutdownChore(regionsRecoveryChore);
}
}

View File

@ -23,7 +23,6 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.HConstants;
@ -70,7 +69,6 @@ public class RegionsRecoveryChore extends ScheduledChore {
*/
RegionsRecoveryChore(final Stoppable stopper, final Configuration configuration,
final HMaster hMaster) {
super(REGIONS_RECOVERY_CHORE_NAME, stopper, configuration.getInt(
HConstants.REGIONS_RECOVERY_INTERVAL, HConstants.DEFAULT_REGIONS_RECOVERY_INTERVAL));
this.hMaster = hMaster;
@ -125,7 +123,6 @@ public class RegionsRecoveryChore extends ScheduledChore {
private Map<TableName, List<byte[]>> getTableToRegionsByRefCount(
final Map<ServerName, ServerMetrics> serverMetricsMap) {
final Map<TableName, List<byte[]>> tableToReopenRegionsMap = new HashMap<>();
for (ServerMetrics serverMetrics : serverMetricsMap.values()) {
Map<byte[], RegionMetrics> regionMetricsMap = serverMetrics.getRegionMetrics();
@ -146,13 +143,11 @@ public class RegionsRecoveryChore extends ScheduledChore {
}
}
return tableToReopenRegionsMap;
}
private void prepareTableToReopenRegionsMap(
final Map<TableName, List<byte[]>> tableToReopenRegionsMap,
final byte[] regionName, final int regionStoreRefCount) {
final RegionInfo regionInfo = hMaster.getAssignmentManager().getRegionInfo(regionName);
final TableName tableName = regionInfo.getTable();
if (TableName.isMetaTableName(tableName)) {
@ -165,21 +160,4 @@ public class RegionsRecoveryChore extends ScheduledChore {
tableToReopenRegionsMap
.computeIfAbsent(tableName, (key) -> new ArrayList<>()).add(regionName);
}
// hashcode/equals implementation to ensure at-most one object of RegionsRecoveryChore
// is scheduled at a time - RegionsRecoveryConfigManager
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
return o != null && getClass() == o.getClass();
}
@Override
public int hashCode() {
return 31;
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.master;
import com.google.errorprone.annotations.RestrictedApi;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.HConstants;
@ -27,8 +28,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Config manager for RegionsRecovery Chore - Dynamically reload config and update chore
* accordingly
* Config manager for RegionsRecovery Chore - Dynamically reload config and update chore accordingly
*/
@InterfaceAudience.Private
public class RegionsRecoveryConfigManager implements ConfigurationObserver {
@ -36,6 +36,7 @@ public class RegionsRecoveryConfigManager implements ConfigurationObserver {
private static final Logger LOG = LoggerFactory.getLogger(RegionsRecoveryConfigManager.class);
private final HMaster hMaster;
private RegionsRecoveryChore chore;
private int prevMaxStoreFileRefCount;
private int prevRegionsRecoveryInterval;
@ -51,34 +52,35 @@ public class RegionsRecoveryConfigManager implements ConfigurationObserver {
final int newMaxStoreFileRefCount = getMaxStoreFileRefCount(conf);
final int newRegionsRecoveryInterval = getRegionsRecoveryChoreInterval(conf);
if (prevMaxStoreFileRefCount == newMaxStoreFileRefCount
&& prevRegionsRecoveryInterval == newRegionsRecoveryInterval) {
if (prevMaxStoreFileRefCount == newMaxStoreFileRefCount &&
prevRegionsRecoveryInterval == newRegionsRecoveryInterval) {
// no need to re-schedule the chore with updated config
// as there is no change in desired configs
return;
}
LOG.info("Config Reload for RegionsRecovery Chore. prevMaxStoreFileRefCount: {}," +
LOG.info(
"Config Reload for RegionsRecovery Chore. prevMaxStoreFileRefCount: {}," +
" newMaxStoreFileRefCount: {}, prevRegionsRecoveryInterval: {}, " +
"newRegionsRecoveryInterval: {}", prevMaxStoreFileRefCount, newMaxStoreFileRefCount,
prevRegionsRecoveryInterval, newRegionsRecoveryInterval);
"newRegionsRecoveryInterval: {}",
prevMaxStoreFileRefCount, newMaxStoreFileRefCount, prevRegionsRecoveryInterval,
newRegionsRecoveryInterval);
RegionsRecoveryChore regionsRecoveryChore = new RegionsRecoveryChore(this.hMaster,
conf, this.hMaster);
RegionsRecoveryChore regionsRecoveryChore =
new RegionsRecoveryChore(this.hMaster, conf, this.hMaster);
ChoreService choreService = this.hMaster.getChoreService();
// Regions Reopen based on very high storeFileRefCount is considered enabled
// only if hbase.regions.recovery.store.file.ref.count has value > 0
synchronized (this) {
if (chore != null) {
chore.shutdown();
chore = null;
}
if (newMaxStoreFileRefCount > 0) {
// reschedule the chore
// provide mayInterruptIfRunning - false to take care of completion
// of in progress task if any
choreService.cancelChore(regionsRecoveryChore, false);
// schedule the new chore
choreService.scheduleChore(regionsRecoveryChore);
} else {
choreService.cancelChore(regionsRecoveryChore, false);
chore = regionsRecoveryChore;
}
this.prevMaxStoreFileRefCount = newMaxStoreFileRefCount;
this.prevRegionsRecoveryInterval = newRegionsRecoveryInterval;
@ -86,15 +88,18 @@ public class RegionsRecoveryConfigManager implements ConfigurationObserver {
}
private int getMaxStoreFileRefCount(Configuration configuration) {
return configuration.getInt(
HConstants.STORE_FILE_REF_COUNT_THRESHOLD,
return configuration.getInt(HConstants.STORE_FILE_REF_COUNT_THRESHOLD,
HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD);
}
private int getRegionsRecoveryChoreInterval(Configuration configuration) {
return configuration.getInt(
HConstants.REGIONS_RECOVERY_INTERVAL,
return configuration.getInt(HConstants.REGIONS_RECOVERY_INTERVAL,
HConstants.DEFAULT_REGIONS_RECOVERY_INTERVAL);
}
@RestrictedApi(explanation = "Only visible for testing", link = "",
allowedOnPath = ".*/src/test/.*")
RegionsRecoveryChore getChore() {
return chore;
}
}

View File

@ -936,7 +936,7 @@ public class ServerManager {
*/
public void stop() {
if (flushedSeqIdFlusher != null) {
flushedSeqIdFlusher.cancel();
flushedSeqIdFlusher.shutdown();
}
if (persistFlushedSequenceId) {
try {

View File

@ -456,7 +456,7 @@ public class SplitLogManager {
choreService.shutdown();
}
if (timeoutMonitor != null) {
timeoutMonitor.cancel(true);
timeoutMonitor.shutdown(true);
}
}

View File

@ -102,7 +102,7 @@ public class QuotaCache implements Stoppable {
public void stop(final String why) {
if (refreshChore != null) {
LOG.debug("Stopping QuotaRefresherChore chore.");
refreshChore.cancel(true);
refreshChore.shutdown(true);
}
stopped = true;
}

View File

@ -98,11 +98,11 @@ public class RegionServerSpaceQuotaManager {
public synchronized void stop() {
if (spaceQuotaRefresher != null) {
spaceQuotaRefresher.cancel();
spaceQuotaRefresher.shutdown();
spaceQuotaRefresher = null;
}
if (regionSizeReporter != null) {
regionSizeReporter.cancel();
regionSizeReporter.shutdown();
regionSizeReporter = null;
}
started = false;

View File

@ -2642,6 +2642,11 @@ public class HRegionServer extends Thread implements
}
}
protected final void shutdownChore(ScheduledChore chore) {
if (chore != null) {
chore.shutdown();
}
}
/**
* Wait on all threads to finish. Presumption is that all closes and stops
* have already been called.
@ -2649,15 +2654,16 @@ public class HRegionServer extends Thread implements
protected void stopServiceThreads() {
// clean up the scheduled chores
if (this.choreService != null) {
choreService.cancelChore(nonceManagerChore);
choreService.cancelChore(compactionChecker);
choreService.cancelChore(periodicFlusher);
choreService.cancelChore(healthCheckChore);
choreService.cancelChore(executorStatusChore);
choreService.cancelChore(storefileRefresher);
choreService.cancelChore(fsUtilizationChore);
choreService.cancelChore(slowLogTableOpsChore);
// clean up the remaining scheduled chores (in case we missed out any)
shutdownChore(nonceManagerChore);
shutdownChore(compactionChecker);
shutdownChore(periodicFlusher);
shutdownChore(healthCheckChore);
shutdownChore(executorStatusChore);
shutdownChore(storefileRefresher);
shutdownChore(fsUtilizationChore);
shutdownChore(slowLogTableOpsChore);
// cancel the remaining scheduled chores (in case we missed out any)
// TODO: cancel will not cleanup the chores, so we need make sure we do not miss any
choreService.shutdown();
}

View File

@ -216,7 +216,7 @@ public class HeapMemoryManager {
public void stop() {
// The thread is Daemon. Just interrupting the ongoing process.
LOG.info("Stopping");
this.heapMemTunerChore.cancel(true);
this.heapMemTunerChore.shutdown(true);
}
public void registerTuneObserver(HeapMemoryTuneObserver observer) {

View File

@ -18,18 +18,18 @@
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
@ -38,7 +38,7 @@ import org.junit.experimental.categories.Category;
/**
* Test for Regions Recovery Config Manager
*/
@Category({MasterTests.class, MediumTests.class})
@Category({ MasterTests.class, MediumTests.class })
public class TestRegionsRecoveryConfigManager {
@ClassRule
@ -51,8 +51,6 @@ public class TestRegionsRecoveryConfigManager {
private HMaster hMaster;
private RegionsRecoveryChore regionsRecoveryChore;
private RegionsRecoveryConfigManager regionsRecoveryConfigManager;
private Configuration conf;
@ -62,10 +60,8 @@ public class TestRegionsRecoveryConfigManager {
conf = HBASE_TESTING_UTILITY.getConfiguration();
conf.unset("hbase.regions.recovery.store.file.ref.count");
conf.unset("hbase.master.regions.recovery.check.interval");
StartMiniClusterOption option = StartMiniClusterOption.builder()
.masterClass(TestHMaster.class)
.numRegionServers(1)
.numDataNodes(1).build();
StartMiniClusterOption option = StartMiniClusterOption.builder().masterClass(TestHMaster.class)
.numRegionServers(1).numDataNodes(1).build();
HBASE_TESTING_UTILITY.startMiniCluster(option);
cluster = HBASE_TESTING_UTILITY.getMiniHBaseCluster();
}
@ -77,44 +73,44 @@ public class TestRegionsRecoveryConfigManager {
@Test
public void testChoreSchedule() throws Exception {
this.hMaster = cluster.getMaster();
Stoppable stoppable = new StoppableImplementation();
this.regionsRecoveryChore = new RegionsRecoveryChore(stoppable, conf, hMaster);
this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this.hMaster);
// not yet scheduled
Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertFalse(
hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
this.regionsRecoveryConfigManager.onConfigurationChange(conf);
// not yet scheduled
Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertFalse(
hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
conf.setInt("hbase.master.regions.recovery.check.interval", 10);
this.regionsRecoveryConfigManager.onConfigurationChange(conf);
// not yet scheduled - missing config: hbase.regions.recovery.store.file.ref.count
Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertFalse(
hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
conf.setInt("hbase.regions.recovery.store.file.ref.count", 10);
this.regionsRecoveryConfigManager.onConfigurationChange(conf);
// chore scheduled
Assert.assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
conf.setInt("hbase.regions.recovery.store.file.ref.count", 20);
this.regionsRecoveryConfigManager.onConfigurationChange(conf);
// chore re-scheduled
Assert.assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
conf.setInt("hbase.regions.recovery.store.file.ref.count", 20);
this.regionsRecoveryConfigManager.onConfigurationChange(conf);
// chore scheduling untouched
Assert.assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertTrue(hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
conf.unset("hbase.regions.recovery.store.file.ref.count");
this.regionsRecoveryConfigManager.onConfigurationChange(conf);
// chore un-scheduled
Assert.assertFalse(hMaster.getChoreService().isChoreScheduled(regionsRecoveryChore));
assertFalse(
hMaster.getChoreService().isChoreScheduled(regionsRecoveryConfigManager.getChore()));
}
// Make it public so that JVMClusterUtil can access it.
@ -123,24 +119,4 @@ public class TestRegionsRecoveryConfigManager {
super(conf);
}
}
/**
* Simple helper class that just keeps track of whether or not its stopped.
*/
private static class StoppableImplementation implements Stoppable {
private boolean stop = false;
@Override
public void stop(String why) {
this.stop = true;
}
@Override
public boolean isStopped() {
return this.stop;
}
}
}

View File

@ -111,7 +111,7 @@ public class TestCatalogJanitor {
@After
public void teardown() {
this.janitor.cancel(true);
this.janitor.shutdown(true);
this.masterServices.stop("DONE");
}