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
public class CompatibilitySingletonFactory extends CompatibilityFactory {
@SuppressWarnings("ImmutableEnumChecker")
public static enum SingletonStorage {
INSTANCE;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -116,13 +116,13 @@ public interface MetricsRegionWrapper {
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
*/
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
*/
long getNumFlushesQueued();
@ -150,7 +150,7 @@ public interface MetricsRegionWrapper {
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
*/
long getMaxCompactedStoreFileRefCount();

View File

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

View File

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

View File

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

View File

@ -22,10 +22,11 @@ import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsReplicationSourceFactoryImpl implements MetricsReplicationSourceFactory {
@SuppressWarnings("ImmutableEnumChecker")
private static enum SourceHolder {
INSTANCE;
final MetricsReplicationSourceImpl source = new MetricsReplicationSourceImpl();
private final MetricsReplicationSourceImpl source = new MetricsReplicationSourceImpl();
}
@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
* created.
*/
@SuppressWarnings("ImmutableEnumChecker")
private enum FactoryStorage {
INSTANCE;
MetricsThriftServerSourceImpl thriftOne;
MetricsThriftServerSourceImpl thriftTwo;
private MetricsThriftServerSourceImpl thriftOne;
private MetricsThriftServerSourceImpl thriftTwo;
}
@Override

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -30,6 +30,7 @@ public interface HadoopShims {
* TaskAttemptId.forName()
* @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
*/
@SuppressWarnings("TypeParameterUnusedInFormals")
<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
*/
@Override
@SuppressWarnings("unchecked")
@SuppressWarnings({ "unchecked", "TypeParameterUnusedInFormals" })
public <T, J> T createTestTaskAttemptContext(J job, String taskId) {
Job j = (Job) job;
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 class TestCompatibilitySingletonFactoryCallable implements Callable<String> {
private static class TestCompatibilitySingletonFactoryCallable implements Callable<String> {
@Override
public String call() throws Exception {