HBASE-27235 Clean up error-prone warnings in hbase-hadoop-compat (#4648)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
Andrew Purtell 2022-07-25 17:30:15 -07:00 committed by GitHub
parent e10c15d030
commit ca545c4f98
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 55 additions and 31 deletions

View File

@ -31,6 +31,8 @@ import org.slf4j.LoggerFactory;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class CompatibilitySingletonFactory extends CompatibilityFactory { public class CompatibilitySingletonFactory extends CompatibilityFactory {
@SuppressWarnings("ImmutableEnumChecker")
public static enum SingletonStorage { public static enum SingletonStorage {
INSTANCE; INSTANCE;

View File

@ -23,10 +23,12 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsHBaseServerSourceFactoryImpl extends MetricsHBaseServerSourceFactory { public class MetricsHBaseServerSourceFactoryImpl extends MetricsHBaseServerSourceFactory {
@SuppressWarnings("ImmutableEnumChecker")
private enum SourceStorage { private enum SourceStorage {
INSTANCE; INSTANCE;
HashMap<String, MetricsHBaseServerSource> sources = new HashMap<>(); private final HashMap<String, MetricsHBaseServerSource> sources = new HashMap<>();
} }
@Override @Override

View File

@ -24,8 +24,6 @@ import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.JobSubmissionFiles; import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability; import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* Utility methods to interact with a job. * Utility methods to interact with a job.
@ -33,7 +31,6 @@ import org.slf4j.LoggerFactory;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public abstract class JobUtil { public abstract class JobUtil {
private static final Logger LOG = LoggerFactory.getLogger(JobUtil.class);
protected JobUtil() { protected JobUtil() {
super(); super();

View File

@ -66,6 +66,7 @@ public class MetricsAssignmentManagerSourceImpl extends BaseSourceImpl
super(metricsName, metricsDescription, metricsContext, metricsJmxContext); super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
} }
@Override
public void init() { public void init() {
ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, RIT_COUNT_DESC, 0L); ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, RIT_COUNT_DESC, 0L);
ritCountOverThresholdGauge = ritCountOverThresholdGauge =

View File

@ -24,10 +24,12 @@ import org.apache.yetus.audience.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsMasterSourceFactoryImpl implements MetricsMasterSourceFactory { public class MetricsMasterSourceFactoryImpl implements MetricsMasterSourceFactory {
@SuppressWarnings("ImmutableEnumChecker")
private static enum FactoryStorage { private static enum FactoryStorage {
INSTANCE; INSTANCE;
MetricsMasterSource masterSource; private MetricsMasterSource masterSource;
} }
@Override @Override

View File

@ -64,6 +64,7 @@ public class MetricsStochasticBalancerSourceImpl extends MetricsBalancerSourceIm
/** /**
* Reports stochastic load balancer costs to JMX * Reports stochastic load balancer costs to JMX
*/ */
@Override
public void updateStochasticCost(String tableName, String costFunctionName, String functionDesc, public void updateStochasticCost(String tableName, String costFunctionName, String functionDesc,
Double cost) { Double cost) {
if (tableName == null || costFunctionName == null || cost == null) { if (tableName == null || costFunctionName == null || cost == null) {

View File

@ -39,6 +39,7 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class BaseSourceImpl implements BaseSource, MetricsSource { public class BaseSourceImpl implements BaseSource, MetricsSource {
@SuppressWarnings("ImmutableEnumChecker")
private static enum DefaultMetricsSystemInitializer { private static enum DefaultMetricsSystemInitializer {
INSTANCE; INSTANCE;
@ -121,6 +122,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
} }
@Override
public void init() { public void init() {
this.metricsRegistry.clearMetrics(); this.metricsRegistry.clearMetrics();
} }
@ -130,6 +132,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param gaugeName gauge name * @param gaugeName gauge name
* @param value the new value of the gauge. * @param value the new value of the gauge.
*/ */
@Override
public void setGauge(String gaugeName, long value) { public void setGauge(String gaugeName, long value) {
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, value); MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, value);
gaugeInt.set(value); gaugeInt.set(value);
@ -140,6 +143,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param gaugeName The name of the gauge to increment. * @param gaugeName The name of the gauge to increment.
* @param delta The amount to increment the gauge by. * @param delta The amount to increment the gauge by.
*/ */
@Override
public void incGauge(String gaugeName, long delta) { public void incGauge(String gaugeName, long delta) {
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0L); MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0L);
gaugeInt.incr(delta); gaugeInt.incr(delta);
@ -150,6 +154,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param gaugeName The name of the gauge. * @param gaugeName The name of the gauge.
* @param delta the ammount to subtract from a gauge value. * @param delta the ammount to subtract from a gauge value.
*/ */
@Override
public void decGauge(String gaugeName, long delta) { public void decGauge(String gaugeName, long delta) {
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0L); MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0L);
gaugeInt.decr(delta); gaugeInt.decr(delta);
@ -160,6 +165,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param key the name of the counter * @param key the name of the counter
* @param delta the ammount to increment * @param delta the ammount to increment
*/ */
@Override
public void incCounters(String key, long delta) { public void incCounters(String key, long delta) {
MutableFastCounter counter = metricsRegistry.getCounter(key, 0L); MutableFastCounter counter = metricsRegistry.getCounter(key, 0L);
counter.incr(delta); counter.incr(delta);
@ -176,6 +182,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* Remove a named gauge. * Remove a named gauge.
* @param key the key of the gauge to remove * @param key the key of the gauge to remove
*/ */
@Override
public void removeMetric(String key) { public void removeMetric(String key) {
metricsRegistry.removeMetric(key); metricsRegistry.removeMetric(key);
JmxCacheBuster.clearJmxCache(); JmxCacheBuster.clearJmxCache();
@ -190,18 +197,22 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
return metricsRegistry; return metricsRegistry;
} }
@Override
public String getMetricsContext() { public String getMetricsContext() {
return metricsContext; return metricsContext;
} }
@Override
public String getMetricsDescription() { public String getMetricsDescription() {
return metricsDescription; return metricsDescription;
} }
@Override
public String getMetricsJmxContext() { public String getMetricsJmxContext() {
return metricsJmxContext; return metricsJmxContext;
} }
@Override
public String getMetricsName() { public String getMetricsName() {
return metricsName; return metricsName;
} }

View File

@ -39,6 +39,7 @@ public final class Interns {
private static LoadingCache<String, ConcurrentHashMap<String, MetricsInfo>> infoCache = private static LoadingCache<String, ConcurrentHashMap<String, MetricsInfo>> infoCache =
CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.DAYS) CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.DAYS)
.build(new CacheLoader<String, ConcurrentHashMap<String, MetricsInfo>>() { .build(new CacheLoader<String, ConcurrentHashMap<String, MetricsInfo>>() {
@Override
public ConcurrentHashMap<String, MetricsInfo> load(String key) { public ConcurrentHashMap<String, MetricsInfo> load(String key) {
return new ConcurrentHashMap<>(); return new ConcurrentHashMap<>();
} }
@ -46,6 +47,7 @@ public final class Interns {
private static LoadingCache<MetricsInfo, ConcurrentHashMap<String, MetricsTag>> tagCache = private static LoadingCache<MetricsInfo, ConcurrentHashMap<String, MetricsTag>> tagCache =
CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.DAYS) CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.DAYS)
.build(new CacheLoader<MetricsInfo, ConcurrentHashMap<String, MetricsTag>>() { .build(new CacheLoader<MetricsInfo, ConcurrentHashMap<String, MetricsTag>>() {
@Override
public ConcurrentHashMap<String, MetricsTag> load(MetricsInfo key) { public ConcurrentHashMap<String, MetricsTag> load(MetricsInfo key) {
return new ConcurrentHashMap<>(); return new ConcurrentHashMap<>();
} }

View File

@ -52,6 +52,7 @@ public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl
// Every few mins clean the JMX cache. // Every few mins clean the JMX cache.
executor.getExecutor().scheduleWithFixedDelay(new Runnable() { executor.getExecutor().scheduleWithFixedDelay(new Runnable() {
@Override
public void run() { public void run() {
JmxCacheBuster.clearJmxCache(); JmxCacheBuster.clearJmxCache();
} }

View File

@ -27,10 +27,12 @@ import org.apache.yetus.audience.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServerSourceFactory { public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServerSourceFactory {
@SuppressWarnings("ImmutableEnumChecker")
public static enum FactoryStorage { public static enum FactoryStorage {
INSTANCE; INSTANCE;
private Object aggLock = new Object(); private final Object aggLock = new Object();
private MetricsRegionAggregateSourceImpl regionAggImpl; private MetricsRegionAggregateSourceImpl regionAggImpl;
private MetricsUserAggregateSourceImpl userAggImpl; private MetricsUserAggregateSourceImpl userAggImpl;
private MetricsTableAggregateSourceImpl tblAggImpl; private MetricsTableAggregateSourceImpl tblAggImpl;
@ -46,6 +48,7 @@ public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServer
} }
} }
@Override
public synchronized MetricsUserAggregateSourceImpl getUserAggregate() { public synchronized MetricsUserAggregateSourceImpl getUserAggregate() {
synchronized (FactoryStorage.INSTANCE.aggLock) { synchronized (FactoryStorage.INSTANCE.aggLock) {
if (FactoryStorage.INSTANCE.userAggImpl == null) { if (FactoryStorage.INSTANCE.userAggImpl == null) {
@ -91,6 +94,7 @@ public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServer
return new MetricsTableSourceImpl(table, getTableAggregate(), wrapper); return new MetricsTableSourceImpl(table, getTableAggregate(), wrapper);
} }
@Override
public MetricsIOSource createIO(MetricsIOWrapper wrapper) { public MetricsIOSource createIO(MetricsIOWrapper wrapper) {
return new MetricsIOSourceImpl(wrapper); return new MetricsIOSourceImpl(wrapper);
} }

View File

@ -116,14 +116,14 @@ public interface MetricsRegionWrapper {
long getNumCompactionsFailed(); long getNumCompactionsFailed();
/** /**
* @return the total number of compactions that are currently queued(or being executed) at point * Returns the total number of compactions that are currently queued(or being executed) at point
* in time * in time
*/ */
long getNumCompactionsQueued(); long getNumCompactionsQueued();
/** /**
* @return the total number of flushes currently queued(being executed) for this region at point * Returns the total number of flushes currently queued(being executed) for this region at point
* in time * in time
*/ */
long getNumFlushesQueued(); long getNumFlushesQueued();
@ -150,8 +150,8 @@ public interface MetricsRegionWrapper {
long getStoreRefCount(); long getStoreRefCount();
/** /**
* @return the max number of references active on any store file among all compacted store files * Returns the max number of references active on any store file among all compacted store files
* that belong to this region * that belong to this region
*/ */
long getMaxCompactedStoreFileRefCount(); long getMaxCompactedStoreFileRefCount();

View File

@ -43,10 +43,6 @@ public class MetricsTableAggregateSourceImpl extends BaseSourceImpl
super(metricsName, metricsDescription, metricsContext, metricsJmxContext); super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
} }
private void register(MetricsTableSource source) {
source.registerMetrics();
}
@Override @Override
public void deleteTableSource(String table) { public void deleteTableSource(String table) {
try { try {

View File

@ -74,6 +74,9 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsTableSourceImpl implements MetricsTableSource { public class MetricsTableSourceImpl implements MetricsTableSource {
@ -354,8 +357,9 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
for (Entry<String, Long> entry : metricMap.entrySet()) { for (Entry<String, Long> entry : metricMap.entrySet()) {
// append 'store' and its name to the metric // append 'store' and its name to the metric
mrb.addGauge(Interns.info(this.tableNamePrefixPart1 + _COLUMNFAMILY mrb.addGauge(Interns.info(this.tableNamePrefixPart1 + _COLUMNFAMILY
+ entry.getKey().split(MetricsTableWrapperAggregate.HASH)[1] + this.tableNamePrefixPart2 + Iterables
+ metricName, metricDesc), entry.getValue()); .get(Splitter.onPattern(MetricsTableWrapperAggregate.HASH).split(entry.getKey()), 1)
+ this.tableNamePrefixPart2 + metricName, metricDesc), entry.getValue());
} }
} }
} }
@ -375,11 +379,9 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
if (this == o) { if (this == o) {
return true; return true;
} }
if (!(o instanceof MetricsTableSourceImpl)) {
if (o == null || getClass() != o.getClass()) {
return false; return false;
} }
return (compareTo((MetricsTableSourceImpl) o) == 0); return (compareTo((MetricsTableSourceImpl) o) == 0);
} }

View File

@ -57,7 +57,6 @@ public class MetricsUserSourceImpl implements MetricsUserSource {
private final int hashCode; private final int hashCode;
private AtomicBoolean closed = new AtomicBoolean(false); private AtomicBoolean closed = new AtomicBoolean(false);
private final MetricsUserAggregateSourceImpl agg;
private final DynamicMetricsRegistry registry; private final DynamicMetricsRegistry registry;
private ConcurrentHashMap<String, ClientMetrics> clientMetricsMap; private ConcurrentHashMap<String, ClientMetrics> clientMetricsMap;
@ -115,7 +114,6 @@ public class MetricsUserSourceImpl implements MetricsUserSource {
} }
this.user = user; this.user = user;
this.agg = agg;
this.registry = agg.getMetricsRegistry(); this.registry = agg.getMetricsRegistry();
this.userNamePrefix = "user_" + user + "_metric_"; this.userNamePrefix = "user_" + user + "_metric_";

View File

@ -22,10 +22,11 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsReplicationSourceFactoryImpl implements MetricsReplicationSourceFactory { public class MetricsReplicationSourceFactoryImpl implements MetricsReplicationSourceFactory {
@SuppressWarnings("ImmutableEnumChecker")
private static enum SourceHolder { private static enum SourceHolder {
INSTANCE; INSTANCE;
final MetricsReplicationSourceImpl source = new MetricsReplicationSourceImpl(); private final MetricsReplicationSourceImpl source = new MetricsReplicationSourceImpl();
} }
@Override @Override

View File

@ -29,11 +29,12 @@ public class MetricsThriftServerSourceFactoryImpl implements MetricsThriftServer
* A singleton used to make sure that only one thrift metrics source per server type is ever * A singleton used to make sure that only one thrift metrics source per server type is ever
* created. * created.
*/ */
@SuppressWarnings("ImmutableEnumChecker")
private enum FactoryStorage { private enum FactoryStorage {
INSTANCE; INSTANCE;
MetricsThriftServerSourceImpl thriftOne; private MetricsThriftServerSourceImpl thriftOne;
MetricsThriftServerSourceImpl thriftTwo; private MetricsThriftServerSourceImpl thriftTwo;
} }
@Override @Override

View File

@ -83,6 +83,7 @@ public class MetricsZooKeeperSourceImpl extends BaseSourceImpl implements Metric
SYNC_OPERATION_LATENCY_DESC); SYNC_OPERATION_LATENCY_DESC);
} }
@Override
public void getMetrics(MetricsCollector metricsCollector, boolean all) { public void getMetrics(MetricsCollector metricsCollector, boolean all) {
super.getMetrics(metricsCollector, all); super.getMetrics(metricsCollector, all);
clearZKExceptionMetrics(); clearZKExceptionMetrics();

View File

@ -33,6 +33,7 @@ public class DefaultMetricsSystemHelper {
private final Field mapField; private final Field mapField;
public DefaultMetricsSystemHelper() { public DefaultMetricsSystemHelper() {
@SuppressWarnings("GetClassOnEnum")
Class<? extends DefaultMetricsSystem> clazz = DefaultMetricsSystem.INSTANCE.getClass(); Class<? extends DefaultMetricsSystem> clazz = DefaultMetricsSystem.INSTANCE.getClass();
Method m; Method m;
try { try {

View File

@ -26,8 +26,6 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.MetricsTag; import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.impl.MsInfo; import org.apache.hadoop.metrics2.impl.MsInfo;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
@ -43,7 +41,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class DynamicMetricsRegistry { public class DynamicMetricsRegistry {
private static final Logger LOG = LoggerFactory.getLogger(DynamicMetricsRegistry.class);
private final ConcurrentMap<String, MutableMetric> metricsMap = Maps.newConcurrentMap(); private final ConcurrentMap<String, MutableMetric> metricsMap = Maps.newConcurrentMap();
private final ConcurrentMap<String, MetricsTag> tagsMap = Maps.newConcurrentMap(); private final ConcurrentMap<String, MetricsTag> tagsMap = Maps.newConcurrentMap();

View File

@ -43,6 +43,7 @@ public class MetricsExecutorImpl implements MetricsExecutor {
} }
} }
@SuppressWarnings("ImmutableEnumChecker")
private enum ExecutorSingleton { private enum ExecutorSingleton {
INSTANCE; INSTANCE;

View File

@ -46,6 +46,7 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
this.histogram = new HistogramImpl(); this.histogram = new HistogramImpl();
} }
@Override
public void add(final long val) { public void add(final long val) {
histogram.update(val); histogram.update(val);
} }

View File

@ -35,6 +35,7 @@ import org.apache.yetus.audience.InterfaceAudience;
* summaries" in SIGMOD 2001 * summaries" in SIGMOD 2001
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@SuppressWarnings("JdkObsolete") // This is a use case for LinkedList
public class MetricSampleQuantiles { public class MetricSampleQuantiles {
/** /**

View File

@ -30,6 +30,7 @@ public interface HadoopShims {
* TaskAttemptId.forName() * TaskAttemptId.forName()
* @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext * @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
*/ */
@SuppressWarnings("TypeParameterUnusedInFormals")
<T, J> T createTestTaskAttemptContext(final J job, final String taskId); <T, J> T createTestTaskAttemptContext(final J job, final String taskId);
} }

View File

@ -33,7 +33,7 @@ public class HadoopShimsImpl implements HadoopShims {
* @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext * @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
*/ */
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings({ "unchecked", "TypeParameterUnusedInFormals" })
public <T, J> T createTestTaskAttemptContext(J job, String taskId) { public <T, J> T createTestTaskAttemptContext(J job, String taskId) {
Job j = (Job) job; Job j = (Job) job;
return (T) new TaskAttemptContextImpl(j.getConfiguration(), TaskAttemptID.forName(taskId)); return (T) new TaskAttemptContextImpl(j.getConfiguration(), TaskAttemptID.forName(taskId));

View File

@ -41,7 +41,7 @@ public class TestCompatibilitySingletonFactory {
private static final int ITERATIONS = 100000; private static final int ITERATIONS = 100000;
private class TestCompatibilitySingletonFactoryCallable implements Callable<String> { private static class TestCompatibilitySingletonFactoryCallable implements Callable<String> {
@Override @Override
public String call() throws Exception { public String call() throws Exception {