HBASE-24451 Remove the HasThread because the related bug had been fixed since jdk7 (#1797)

Signed-off-by: stack <stack@apache.org>
Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
bsglz 2020-05-29 20:03:18 +08:00 committed by GitHub
parent fe1fc25fba
commit 7b7e25a50b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 30 additions and 139 deletions

View File

@ -242,4 +242,9 @@
<Bug pattern="MS_EXPOSE_REP"/>
</Match>
<Match>
<Class name="org.apache.hadoop.hbase.io.hfile.LruBlockCache"/>
<Bug pattern="SC_START_IN_CTOR"/>
</Match>
</FindBugsFilter>

View File

@ -1,103 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import java.lang.Thread.UncaughtExceptionHandler;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Abstract class which contains a Thread and delegates the common Thread
* methods to that instance.
*
* The purpose of this class is to workaround Sun JVM bug #6915621, in which
* something internal to the JDK uses Thread.currentThread() as a monitor
* lock. This can produce deadlocks like HBASE-4367, HBASE-4101, etc.
*/
@InterfaceAudience.Private
public abstract class HasThread implements Runnable {
private final Thread thread;
public HasThread() {
this.thread = new Thread(this);
this.thread.setDaemon(true);
}
public HasThread(String name) {
this.thread = new Thread(this, name);
this.thread.setDaemon(true);
}
public Thread getThread() {
return thread;
}
@Override
public abstract void run();
//// Begin delegation to Thread
public final String getName() {
return thread.getName();
}
public void interrupt() {
thread.interrupt();
}
public final boolean isAlive() {
return thread.isAlive();
}
public boolean isInterrupted() {
return thread.isInterrupted();
}
public final void setDaemon(boolean on) {
thread.setDaemon(on);
}
public final void setName(String name) {
thread.setName(name);
}
public final void setPriority(int newPriority) {
thread.setPriority(newPriority);
}
public void setUncaughtExceptionHandler(UncaughtExceptionHandler eh) {
thread.setUncaughtExceptionHandler(eh);
}
public void start() {
thread.start();
}
public final void join() throws InterruptedException {
thread.join();
}
public final void join(long millis, int nanos) throws InterruptedException {
thread.join(millis, nanos);
}
public final void join(long millis) throws InterruptedException {
thread.join(millis);
}
//// End delegation to Thread
}

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -920,7 +919,7 @@ public class LruBlockCache implements FirstLevelBlockCache {
*
* Thread is triggered into action by {@link LruBlockCache#runEviction()}
*/
static class EvictionThread extends HasThread {
static class EvictionThread extends Thread {
private WeakReference<LruBlockCache> cache;
private volatile boolean go = true;

View File

@ -70,7 +70,6 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.nio.RefCnt;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.IdReadWriteLock;
import org.apache.hadoop.hbase.util.IdReadWriteLockStrongRef;
import org.apache.hadoop.hbase.util.IdReadWriteLockWithObjectPool;
@ -878,7 +877,7 @@ public class BucketCache implements BlockCache, HeapSize {
// This handles flushing the RAM cache to IOEngine.
@VisibleForTesting
class WriterThread extends HasThread {
class WriterThread extends Thread {
private final BlockingQueue<RAMQueueEntry> inputQueue;
private volatile boolean writerEnabled = true;

View File

@ -199,7 +199,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
@ -261,7 +260,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* Protection against zombie master. Started once Master accepts active responsibility and
* starts taking over responsibilities. Allows a finite time window before giving up ownership.
*/
private static class InitializationMonitor extends HasThread {
private static class InitializationMonitor extends Thread {
/** The amount of time in milliseconds to sleep before checking initialization status. */
public static final String TIMEOUT_KEY = "hbase.master.initializationmonitor.timeout";
public static final long TIMEOUT_DEFAULT = TimeUnit.MILLISECONDS.convert(15, TimeUnit.MINUTES);

View File

@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -122,9 +121,9 @@ public class SplitLogManager {
throws IOException {
this.server = master;
this.conf = conf;
// Get Server Thread name. Sometimes the Server is mocked so may not implement HasThread.
// Get Server Thread name. Sometimes the Server is mocked so may not extends Thread.
// For example, in tests.
String name = master instanceof HasThread? ((HasThread)master).getName():
String name = master instanceof Thread? ((Thread)master).getName():
master.getServerName().toShortString();
this.choreService =
new ChoreService(name + ".splitLogManager.");

View File

@ -75,7 +75,6 @@ import org.apache.hadoop.hbase.regionserver.SequenceId;
import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
@ -1908,9 +1907,9 @@ public class AssignmentManager {
}
private void startAssignmentThread() {
// Get Server Thread name. Sometimes the Server is mocked so may not implement HasThread.
// Get Server Thread name. Sometimes the Server is mocked so may not implement Thread.
// For example, in tests.
String name = master instanceof HasThread? ((HasThread)master).getName():
String name = master instanceof Thread? ((Thread)master).getName():
master.getServerName().toShortString();
assignThread = new Thread(name) {
@Override

View File

@ -24,7 +24,6 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
import static org.apache.hadoop.hbase.util.DNS.RS_HOSTNAME_KEY;
import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler;
import java.lang.management.MemoryType;
import java.lang.management.MemoryUsage;
import java.lang.reflect.Constructor;
@ -120,7 +119,7 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.MasterRpcServicesVersionWrapper;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.mob.MobFileCache;
import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
@ -161,7 +160,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
import org.apache.hadoop.hbase.util.Pair;
@ -246,7 +244,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
@SuppressWarnings({ "deprecation"})
public class HRegionServer extends HasThread implements
public class HRegionServer extends Thread implements
RegionServerServices, LastSequenceId, ConfigurationObserver {
private static final Logger LOG = LoggerFactory.getLogger(HRegionServer.class);
@ -1461,7 +1459,7 @@ public class HRegionServer extends HasThread implements
LOG.debug("Waiting on {}", this.regionsInTransitionInRS.keySet().stream().
map(e -> Bytes.toString(e)).collect(Collectors.joining(", ")));
}
if (sleep(200)) {
if (sleepInterrupted(200)) {
interrupted = true;
}
}
@ -1472,7 +1470,7 @@ public class HRegionServer extends HasThread implements
}
}
private static boolean sleep(long millis) {
private static boolean sleepInterrupted(long millis) {
boolean interrupted = false;
try {
Thread.sleep(millis);
@ -2004,7 +2002,7 @@ public class HRegionServer extends HasThread implements
this.executorService.startExecutorService(ExecutorType.RS_SWITCH_RPC_THROTTLE,
conf.getInt("hbase.regionserver.executor.switch.rpc.throttle.threads", 1));
Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
Threads.setDaemonThreadRunning(this.walRoller, getName() + ".logRoller",
uncaughtExceptionHandler);
if (this.cacheFlusher != null) {
this.cacheFlusher.start(uncaughtExceptionHandler);
@ -2042,7 +2040,7 @@ public class HRegionServer extends HasThread implements
// Leases is not a Thread. Internally it runs a daemon thread. If it gets
// an unhandled exception, it will just exit.
Threads.setDaemonThreadRunning(this.leaseManager.getThread(), getName() + ".leaseChecker",
Threads.setDaemonThreadRunning(this.leaseManager, getName() + ".leaseChecker",
uncaughtExceptionHandler);
// Create the log splitting worker and start it
@ -2328,7 +2326,7 @@ public class HRegionServer extends HasThread implements
if (hris[0].isMetaRegion()) {
try {
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName,
hris[0].getReplicaId(),State.OPEN);
hris[0].getReplicaId(), RegionState.State.OPEN);
} catch (KeeperException e) {
LOG.info("Failed to update meta location", e);
return false;
@ -2675,7 +2673,7 @@ public class HRegionServer extends HasThread implements
previousLogTime = System.currentTimeMillis();
}
refresh = true; // let's try pull it from ZK directly
if (sleep(200)) {
if (sleepInterrupted(200)) {
interrupted = true;
}
continue;
@ -2707,7 +2705,7 @@ public class HRegionServer extends HasThread implements
}
previousLogTime = System.currentTimeMillis();
}
if (sleep(200)) {
if (sleepInterrupted(200)) {
interrupted = true;
}
}

View File

@ -28,7 +28,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -53,7 +52,7 @@ import org.slf4j.LoggerFactory;
* sleep time which is invariant.
*/
@InterfaceAudience.Private
public class LeaseManager extends HasThread {
public class LeaseManager extends Thread {
private static final Logger LOG = LoggerFactory.getLogger(LeaseManager.class.getName());
private static final int MIN_WAIT_TIME = 100;

View File

@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.ipc.RemoteException;
@ -320,7 +319,7 @@ class MemStoreFlusher implements FlushRequester {
return r == null? 0: r.getMemStoreDataSize();
}
private class FlushHandler extends HasThread {
private class FlushHandler extends Thread {
private FlushHandler(String name) {
super(name);
@ -534,7 +533,7 @@ class MemStoreFlusher implements FlushRequester {
void join() {
for (FlushHandler flushHander : flushHandlers) {
if (flushHander != null) {
Threads.shutdown(flushHander.getThread());
Threads.shutdown(flushHander);
}
}
}

View File

@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.FSHLogProvider;
import org.apache.hadoop.hbase.wal.WALEdit;
@ -453,7 +452,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
* SyncFutures are 'artificial', something to hold the Handler until the filesystem sync
* completes.
*/
private class SyncRunner extends HasThread {
private class SyncRunner extends Thread {
private volatile long sequence;
// Keep around last exception thrown. Clear on successful sync.
private final BlockingQueue<SyncFuture> syncFutures;

View File

@ -336,7 +336,7 @@ public class JVMClusterUtil {
// The below has been replaced to debug sometime hangs on end of
// tests.
// this.master.join():
Threads.threadDumpingIsAlive(t.master.getThread());
Threads.threadDumpingIsAlive(t.master);
} catch(InterruptedException e) {
LOG.info("Got InterruptedException on shutdown - " +
"not waiting anymore on master ends", e);

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALClosedException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -52,7 +51,7 @@ import org.slf4j.LoggerFactory;
* TODO: change to a pool of threads
*/
@InterfaceAudience.Private
public abstract class AbstractWALRoller<T extends Abortable> extends HasThread
public abstract class AbstractWALRoller<T extends Abortable> extends Thread
implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(AbstractWALRoller.class);
@ -223,7 +222,7 @@ public abstract class AbstractWALRoller<T extends Abortable> extends HasThread
protected abstract void scheduleFlush(String encodedRegionName);
private boolean isWaiting() {
Thread.State state = getThread().getState();
Thread.State state = getState();
return state == Thread.State.WAITING || state == Thread.State.TIMED_WAITING;
}

View File

@ -163,7 +163,7 @@ public class TestRegionAssignedToMultipleRegionServers {
KILL = true;
HMaster activeMaster = UTIL.getMiniHBaseCluster().getMaster();
activeMaster.abort("For testing");
activeMaster.getThread().join();
activeMaster.join();
KILL = false;
// sleep a while to reproduce the problem, as after the fix in HBASE-21472 the execution logic

View File

@ -321,7 +321,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
final Runnable[] benchmarks = new Runnable[numRegions];
final MockRegionServerServices mockServices = new MockRegionServerServices(getConf());
final LogRoller roller = new LogRoller(mockServices);
Threads.setDaemonThreadRunning(roller.getThread(), "WALPerfEval.logRoller");
Threads.setDaemonThreadRunning(roller, "WALPerfEval.logRoller");
try {
for(int i = 0; i < numRegions; i++) {