HBASE-15118 Fix findbugs complaint in hbase-server

This commit is contained in:
stack 2016-01-18 13:02:17 -08:00
parent d9fd87d5a8
commit d6654ea7ea
56 changed files with 363 additions and 269 deletions

View File

@ -30,13 +30,11 @@ public class PrettyPrinter {
NONE
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DM_BOXED_PRIMITIVE_FOR_PARSING",
justification="I don't get what FB is complaining about")
public static String format(final String value, final Unit unit) {
StringBuilder human = new StringBuilder();
switch (unit) {
case TIME_INTERVAL:
human.append(humanReadableTTL(Long.valueOf(value)));
human.append(humanReadableTTL(Long.parseLong(value)));
break;
default:
human.append(value);

View File

@ -60,7 +60,7 @@ public class JMXListener implements Coprocessor {
* only 1 JMX instance is allowed, otherwise there is port conflict even if
* we only load regionserver coprocessor on master
*/
private static JMXConnectorServer jmxCS = null;
private static JMXConnectorServer JMX_CS = null;
public static JMXServiceURL buildJMXServiceURL(int rmiRegistryPort,
int rmiConnectorPort) throws IOException {
@ -137,8 +137,13 @@ public class JMXListener implements Coprocessor {
try {
// Start the JMXListener with the connection string
jmxCS = JMXConnectorServerFactory.newJMXConnectorServer(serviceUrl, jmxEnv, mbs);
jmxCS.start();
synchronized(JMXListener.class) {
if (JMX_CS != null) {
throw new RuntimeException("Started by another thread?");
}
JMX_CS = JMXConnectorServerFactory.newJMXConnectorServer(serviceUrl, jmxEnv, mbs);
JMX_CS.start();
}
LOG.info("ConnectorServer started!");
} catch (IOException e) {
LOG.error("fail to start connector server!", e);
@ -148,10 +153,10 @@ public class JMXListener implements Coprocessor {
public void stopConnectorServer() throws IOException {
synchronized(JMXListener.class) {
if (jmxCS != null) {
jmxCS.stop();
if (JMX_CS != null) {
JMX_CS.stop();
LOG.info("ConnectorServer stopped!");
jmxCS = null;
JMX_CS = null;
}
}
}
@ -186,7 +191,7 @@ public class JMXListener implements Coprocessor {
}
synchronized(JMXListener.class) {
if (jmxCS != null) {
if (JMX_CS != null) {
LOG.info("JMXListener has been started at Registry port " + rmiRegistryPort);
}
else {

View File

@ -175,7 +175,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
return;
}
Task task = findOrCreateOrphanTask(path);
if (task.isOrphan() && (task.incarnation == 0)) {
if (task.isOrphan() && (task.incarnation.get() == 0)) {
LOG.info("resubmitting unassigned orphan task " + path);
// ignore failure to resubmit. The timeout-monitor will handle it later
// albeit in a more crude fashion
@ -228,7 +228,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
version = -1;
}
LOG.info("resubmitting task " + path);
task.incarnation++;
task.incarnation.incrementAndGet();
boolean result = resubmit(this.details.getServerName(), path, version);
if (!result) {
task.heartbeatNoDetails(EnvironmentEdgeManager.currentTime());

View File

@ -140,6 +140,8 @@ public class ZKSplitTransactionCoordination implements SplitTransactionCoordinat
* the node is removed or is not in pending_split state any more, we abort the split.
*/
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intended")
public void waitForSplitTransaction(final RegionServerServices services, Region parent,
HRegionInfo hri_a, HRegionInfo hri_b, SplitTransactionDetails sptd) throws IOException {
ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) sptd;

View File

@ -92,6 +92,8 @@ public class ZkRegionMergeCoordination implements RegionMergeCoordination {
*/
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intended")
public void waitForRegionMergeTransaction(RegionServerServices services,
HRegionInfo mergedRegionInfo, HRegion region_a, HRegion region_b, RegionMergeDetails details)
throws IOException {

View File

@ -78,7 +78,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
private TaskExecutor splitTaskExecutor;
private final Object taskReadyLock = new Object();
volatile int taskReadySeq = 0;
private AtomicInteger taskReadySeq = new AtomicInteger(0);
private volatile String currentTask = null;
private int currentVersion;
private volatile boolean shouldStop = false;
@ -106,7 +106,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
if (path.equals(watcher.splitLogZNode)) {
if (LOG.isTraceEnabled()) LOG.trace("tasks arrived or departed on " + path);
synchronized (taskReadyLock) {
taskReadySeq++;
this.taskReadySeq.incrementAndGet();
taskReadyLock.notify();
}
}
@ -400,14 +400,14 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
* policy puts an upper-limit on the number of simultaneous log splitting that could be happening
* in a cluster.
* <p>
* Synchronization using {@link #taskReadyLock} ensures that it will try to grab every task that
* has been put up
* Synchronization using <code>taskReadyLock</code> ensures that it will try to grab every task
* that has been put up
* @throws InterruptedException
*/
@Override
public void taskLoop() throws InterruptedException {
while (!shouldStop) {
int seq_start = taskReadySeq;
int seq_start = taskReadySeq.get();
List<String> paths = null;
paths = getTaskList();
if (paths == null) {
@ -441,7 +441,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
}
SplitLogCounters.tot_wkr_task_grabing.incrementAndGet();
synchronized (taskReadyLock) {
while (seq_start == taskReadySeq) {
while (seq_start == taskReadySeq.get()) {
taskReadyLock.wait(checkInterval);
if (server != null) {
// check to see if we have stale recovering regions in our internal memory state
@ -527,7 +527,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
@Override
public int getTaskReadySeq() {
return taskReadySeq;
return taskReadySeq.get();
}
@Override

View File

@ -152,6 +152,8 @@ public class JMXJsonServlet extends HttpServlet {
* The servlet response we are creating
*/
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="XSS_REQUEST_PARAMETER_TO_SERVLET_WRITER",
justification="TODO: See HBASE-15122")
public void doGet(HttpServletRequest request, HttpServletResponse response) {
try {
if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {

View File

@ -53,6 +53,8 @@ import org.apache.hadoop.hbase.util.Pair;
* it fallbacks to the archived path.
*/
@InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS",
justification="To be fixed but warning suppressed for now")
public class HFileLink extends FileLink {
private static final Log LOG = LogFactory.getLog(HFileLink.class);

View File

@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.io.hfile;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -37,15 +37,12 @@ import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
* Common functionality needed by all versions of {@link HFile} readers.
*/
@InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
public abstract class AbstractHFileReader
implements HFile.Reader, Configurable {
/** Stream to read from. Does checksum verifications in file system */
protected FSDataInputStream istream; // UUF_UNUSED_PUBLIC_OR_PROTECTED_FIELD
/** The file system stream of the underlying {@link HFile} that
* does not do checksum verification in the file system */
protected FSDataInputStream istreamNoFsChecksum; // UUF_UNUSED_PUBLIC_OR_PROTECTED_FIELD
/** Data block index reader keeping the root data index in memory */
protected HFileBlockIndex.BlockIndexReader dataBlockIndexReader;
@ -289,7 +286,7 @@ public abstract class AbstractHFileReader
protected int currMemstoreTSLen;
protected long currMemstoreTS;
protected int blockFetches;
protected AtomicInteger blockFetches = new AtomicInteger();
protected final HFile.Reader reader;

View File

@ -468,6 +468,8 @@ public class HFile {
* @return an appropriate instance of HFileReader
* @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
justification="Intentional")
private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis,
long size, CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
FixedFileTrailer trailer = null;

View File

@ -83,6 +83,8 @@ import com.google.common.base.Preconditions;
* </ul>
*/
@InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="HE_EQUALS_USE_HASHCODE",
justification="Fix!!! Fine for now bug FIXXXXXXX!!!!")
public class HFileBlock implements Cacheable {
/**

View File

@ -25,7 +25,6 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@ -34,6 +33,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.NoTagsKeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
@ -692,6 +692,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
* @return the next block, or null if there are no more data blocks
* @throws IOException
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
justification="Yeah, unnecessary null check; could do w/ clean up")
protected HFileBlock readNextDataBlock() throws IOException {
long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();
if (block == null)
@ -700,8 +702,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
HFileBlock curBlock = block;
do {
if (curBlock.getOffset() >= lastDataBlockOffset)
if (curBlock.getOffset() >= lastDataBlockOffset) {
return null;
}
if (curBlock.getOffset() < 0) {
throw new IOException("Invalid block file offset: " + block);
@ -943,7 +946,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
blockBuffer = block.getBufferWithoutHeader();
readKeyValueLen();
blockFetches++;
blockFetches.incrementAndGet();
// Reset the next indexed key
this.nextIndexedKey = null;
@ -1205,7 +1208,7 @@ public class HFileReaderV2 extends AbstractHFileReader {
}
seeker.setCurrentBuffer(getEncodedBuffer(newBlock));
blockFetches++;
blockFetches.incrementAndGet();
// Reset the next indexed key
this.nextIndexedKey = null;

View File

@ -321,6 +321,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
private long responseBlockSize = 0;
private boolean retryImmediatelySupported;
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
justification="Can't figure why this complaint is happening... see below")
Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
Message param, CellScanner cellScanner, Connection connection, Responder responder,
long size, TraceInfo tinfo, final InetAddress remoteAddress) {
@ -338,15 +340,18 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
this.isError = false;
this.size = size;
this.tinfo = tinfo;
this.user = connection == null ? null : connection.user;
this.user = connection == null? null: connection.user; // FindBugs: NP_NULL_ON_SOME_PATH
this.remoteAddress = remoteAddress;
this.retryImmediatelySupported = connection.retryImmediatelySupported;
this.retryImmediatelySupported =
connection == null? null: connection.retryImmediatelySupported;
}
/**
* Call is done. Execution happened and we returned results to client. It is now safe to
* cleanup.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="Presume the lock on processing request held by caller is protection enough")
void done() {
if (this.cellBlock != null && reservoir != null) {
// Return buffer to reservoir now we are done with it.
@ -588,7 +593,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
return connection.getVersionInfo();
}
@Override
public boolean isRetryImmediatelySupported() {
return retryImmediatelySupported;
@ -764,6 +768,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="selector access is not synchronized; seems fine but concerned changing " +
"it will have per impact")
public void run() {
LOG.info(getName() + ": starting");
while (running) {
@ -1265,15 +1272,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
private boolean useWrap = false;
// Fake 'call' for failed authorization response
private static final int AUTHORIZATION_FAILED_CALLID = -1;
private final Call authFailedCall =
new Call(AUTHORIZATION_FAILED_CALLID, null, null, null, null, null, this, null, 0, null,
null);
private final Call authFailedCall = new Call(AUTHORIZATION_FAILED_CALLID, null, null, null,
null, null, this, null, 0, null, null);
private ByteArrayOutputStream authFailedResponse =
new ByteArrayOutputStream();
// Fake 'call' for SASL context setup
private static final int SASL_CALLID = -33;
private final Call saslCall =
new Call(SASL_CALLID, this.service, null, null, null, null, this, null, 0, null, null);
private final Call saslCall = new Call(SASL_CALLID, null, null, null, null, null, this, null,
0, null, null);
// was authentication allowed with a fallback to simple auth
private boolean authenticatedWithFallback;
@ -2165,7 +2171,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
@Override
public void refreshAuthManager(PolicyProvider pp) {
public synchronized void refreshAuthManager(PolicyProvider pp) {
// Ignore warnings that this should be accessed in a static way instead of via an instance;
// it'll break if you go via static route.
this.authManager.refresh(this.conf, pp);
@ -2391,7 +2397,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
* @throws org.apache.hadoop.security.authorize.AuthorizationException
* when the client isn't authorized to talk the protocol
*/
public void authorize(UserGroupInformation user, ConnectionHeader connection, InetAddress addr)
public synchronized void authorize(UserGroupInformation user, ConnectionHeader connection,
InetAddress addr)
throws AuthorizationException {
if (authorize) {
Class<?> c = getServiceInterface(services, connection.getServiceName());

View File

@ -32,7 +32,6 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
@ -135,17 +134,18 @@ public class HashTable extends Configured implements Tool {
p.setProperty("endTimestamp", Long.toString(endTime));
}
FSDataOutputStream out = fs.create(path);
p.store(new OutputStreamWriter(out, Charsets.UTF_8), null);
out.close();
try (OutputStreamWriter osw = new OutputStreamWriter(fs.create(path), Charsets.UTF_8)) {
p.store(osw, null);
}
}
void readPropertiesFile(FileSystem fs, Path path) throws IOException {
FSDataInputStream in = fs.open(path);
Properties p = new Properties();
p.load(new InputStreamReader(in, Charsets.UTF_8));
in.close();
try (FSDataInputStream in = fs.open(path)) {
try (InputStreamReader isr = new InputStreamReader(in, Charsets.UTF_8)) {
p.load(isr);
}
}
tableName = p.getProperty("table");
families = p.getProperty("columnFamilies");
batchSize = Long.parseLong(p.getProperty("targetBatchSize"));

View File

@ -139,6 +139,8 @@ public class Import {
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
justification="This is wrong, yes, but we should be purging Writables, not fixing them")
public int compareTo(KeyValueWritableComparable o) {
return cellComparator.compare(this.kv, ((KeyValueWritableComparable)o).kv);
}
@ -244,6 +246,8 @@ public class Import {
/**
* A mapper that just writes out KeyValues.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
justification="Writables are going away and this has been this way forever")
public static class KeyValueImporter extends TableMapper<ImmutableBytesWritable, KeyValue> {
private Map<byte[], byte[]> cfRenameMap;
private Filter filter;

View File

@ -108,7 +108,7 @@ public class ImportTsv extends Configured implements Tool {
* If table didn't exist and was created in dry-run mode, this flag is
* flipped to delete it when MR ends.
*/
private static boolean dryRunTableCreated;
private static boolean DRY_RUN_TABLE_CREATED;
public static class TsvParser {
/**
@ -505,7 +505,9 @@ public class ImportTsv extends Configured implements Tool {
createTable(admin, tableName, columns);
if (isDryRun) {
LOG.warn("Dry run: Table will be deleted at end of dry run.");
dryRunTableCreated = true;
synchronized (ImportTsv.class) {
DRY_RUN_TABLE_CREATED = true;
}
}
} else {
String errorMsg =
@ -617,7 +619,8 @@ public class ImportTsv extends Configured implements Tool {
}
admin.deleteTable(tableName);
} catch (IOException e) {
LOG.error(format("***Dry run: Failed to delete table '%s'.***\n%s", tableName, e.toString()));
LOG.error(format("***Dry run: Failed to delete table '%s'.***%n%s", tableName,
e.toString()));
return;
}
LOG.info(format("Dry run: Deleted table '%s'.", tableName));
@ -659,7 +662,7 @@ public class ImportTsv extends Configured implements Tool {
"input data. Another special column" + TsvParser.TIMESTAMPKEY_COLUMN_SPEC +
" designates that this column should be\n" +
"used as timestamp for each record. Unlike " + TsvParser.ROWKEY_COLUMN_SPEC + ", " +
TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional.\n" +
TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional." + "\n" +
"You must specify at most one column as timestamp key for each imported record.\n" +
"Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
"Note: if you use this option, then '" + TIMESTAMP_CONF_KEY + "' option will be ignored.\n" +
@ -770,10 +773,16 @@ public class ImportTsv extends Configured implements Tool {
// system time
getConf().setLong(TIMESTAMP_CONF_KEY, timstamp);
dryRunTableCreated = false;
Job job = createSubmittableJob(getConf(), otherArgs);
synchronized (ImportTsv.class) {
DRY_RUN_TABLE_CREATED = false;
}
Job job = createSubmittableJob(getConf(), args);
boolean success = job.waitForCompletion(true);
if (dryRunTableCreated) {
boolean delete = false;
synchronized (ImportTsv.class) {
delete = DRY_RUN_TABLE_CREATED;
}
if (delete) {
deleteTable(getConf(), args);
}
return success ? 0 : 1;

View File

@ -110,10 +110,8 @@ public abstract class MultiTableInputFormatBase extends
@Override
public void close() throws IOException {
trr.close();
if (connection != null) {
connection.close();
}
}
@Override
public ImmutableBytesWritable getCurrentKey() throws IOException, InterruptedException {
@ -145,9 +143,7 @@ public abstract class MultiTableInputFormatBase extends
// If there is an exception make sure that all
// resources are closed and released.
trr.close();
if (connection != null) {
connection.close();
}
throw ioe;
}
}

View File

@ -230,6 +230,8 @@ public class MultithreadedTableMapper<K2, V2> extends TableMapper<K2, V2> {
}
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Don't understand why FB is complaining about this one. We do throw exception")
private class MapRunner implements Runnable {
private Mapper<ImmutableBytesWritable, Result, K2,V2> mapper;
private Context subcontext;
@ -280,7 +282,7 @@ public class MultithreadedTableMapper<K2, V2> extends TableMapper<K2, V2> {
Class<?> wrappedMapperClass = Class.forName("org.apache.hadoop.mapreduce.lib.map.WrappedMapper");
Method getMapContext = wrappedMapperClass.getMethod("getMapContext", MapContext.class);
subcontext = (Context) getMapContext.invoke(wrappedMapperClass.newInstance(), mc);
} catch (Exception ee) {
} catch (Exception ee) { // FindBugs: REC_CATCH_EXCEPTION
// rethrow as IOE
throw new IOException(e);
}

View File

@ -111,6 +111,8 @@ implements Configurable {
* org.apache.hadoop.conf.Configuration)
*/
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intentional")
public void setConf(Configuration configuration) {
this.conf = configuration;

View File

@ -924,6 +924,9 @@ public class AssignmentManager extends ZooKeeperListener {
* @param coordination coordination for opening region
* @param ord details about opening region
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
justification="Needs work; says access to ConcurrentHashMaps not ATOMIC!!!")
void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
OpenRegionCoordination.OpenRegionDetails ord) {
if (rt == null) {
@ -1047,9 +1050,11 @@ public class AssignmentManager extends ZooKeeperListener {
// No need to use putIfAbsent, or extra synchronization since
// this whole handleRegion block is locked on the encoded region
// name, and failedOpenTracker is updated only in this block
// FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
failedOpenTracker.put(encodedName, failedOpenCount);
}
if (failedOpenCount.incrementAndGet() >= maximumAttempts) {
// FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
regionStates.updateRegionState(rt, State.FAILED_OPEN);
// remove the tracking info to save memory, also reset
// the count for next open initiative
@ -3637,18 +3642,24 @@ public class AssignmentManager extends ZooKeeperListener {
EventType.RS_ZK_REQUEST_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT);
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(
value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
justification="Modification of Maps not ATOMIC!!!! FIX!!!")
private void onRegionFailedOpen(
final HRegionInfo hri, final ServerName sn) {
String encodedName = hri.getEncodedName();
// FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION Worth fixing!!!
AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
if (failedOpenCount == null) {
failedOpenCount = new AtomicInteger();
// No need to use putIfAbsent, or extra synchronization since
// this whole handleRegion block is locked on the encoded region
// name, and failedOpenTracker is updated only in this block
// FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
failedOpenTracker.put(encodedName, failedOpenCount);
}
if (failedOpenCount.incrementAndGet() >= maximumAttempts && !hri.isMetaRegion()) {
// FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
regionStates.updateRegionState(hri, State.FAILED_OPEN);
// remove the tracking info to save memory, also reset
// the count for next open initiative

View File

@ -94,8 +94,7 @@ public class HMasterCommandLine extends ServerCommandLine {
if (cmd.hasOption("minRegionServers")) {
String val = cmd.getOptionValue("minRegionServers");
getConf().setInt("hbase.regions.server.count.min",
Integer.valueOf(val));
getConf().setInt("hbase.regions.server.count.min", Integer.parseInt(val));
LOG.debug("minRegionServers set to " + val);
}
@ -103,7 +102,7 @@ public class HMasterCommandLine extends ServerCommandLine {
if (cmd.hasOption("minServers")) {
String val = cmd.getOptionValue("minServers");
getConf().setInt("hbase.regions.server.count.min",
Integer.valueOf(val));
Integer.parseInt(val));
LOG.debug("minServers set to " + val);
}
@ -116,13 +115,13 @@ public class HMasterCommandLine extends ServerCommandLine {
// master when we are in local/standalone mode. Useful testing)
if (cmd.hasOption("localRegionServers")) {
String val = cmd.getOptionValue("localRegionServers");
getConf().setInt("hbase.regionservers", Integer.valueOf(val));
getConf().setInt("hbase.regionservers", Integer.parseInt(val));
LOG.debug("localRegionServers set to " + val);
}
// How many masters to startup inside this process; useful testing
if (cmd.hasOption("masters")) {
String val = cmd.getOptionValue("masters");
getConf().setInt("hbase.masters", Integer.valueOf(val));
getConf().setInt("hbase.masters", Integer.parseInt(val));
LOG.debug("masters set to " + val);
}

View File

@ -464,8 +464,6 @@ public class RegionStates {
if (!serverName.equals(oldServerName)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName);
} else {
LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName);
}
addToServerHoldings(serverName, hri);
addToReplicaMapping(hri);

View File

@ -643,7 +643,7 @@ public class SplitLogManager {
public volatile ServerName cur_worker_name;
public volatile TaskBatch batch;
public volatile TerminationStatus status;
public volatile int incarnation;
public volatile AtomicInteger incarnation = new AtomicInteger(0);
public final AtomicInteger unforcedResubmits = new AtomicInteger();
public volatile boolean resubmitThresholdReached;
@ -655,7 +655,6 @@ public class SplitLogManager {
}
public Task() {
incarnation = 0;
last_version = -1;
status = IN_PROGRESS;
setUnassigned();

View File

@ -59,12 +59,15 @@ import com.google.common.collect.Sets;
* {@link org.apache.hadoop.hbase.ZKNamespaceManager}
*/
@InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="TODO: synchronize access on nsTable but it is done in tiers above and this " +
"class is going away/shrinking")
public class TableNamespaceManager {
private static final Log LOG = LogFactory.getLog(TableNamespaceManager.class);
private Configuration conf;
private MasterServices masterServices;
private Table nsTable = null;
private Table nsTable = null; // FindBugs: IS2_INCONSISTENT_SYNC TODO: Access is not synchronized
private ZKNamespaceManager zkNamespaceManager;
private boolean initialized;

View File

@ -552,17 +552,21 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
switch (action.type) {
case NULL: break;
case ASSIGN_REGION:
// FindBugs: Having the assert quietens FB BC_UNCONFIRMED_CAST warnings
assert action instanceof AssignRegionAction: action.getClass();
AssignRegionAction ar = (AssignRegionAction) action;
regionsPerServer[ar.server] = addRegion(regionsPerServer[ar.server], ar.region);
regionMoved(ar.region, -1, ar.server);
break;
case MOVE_REGION:
assert action instanceof MoveRegionAction: action.getClass();
MoveRegionAction mra = (MoveRegionAction) action;
regionsPerServer[mra.fromServer] = removeRegion(regionsPerServer[mra.fromServer], mra.region);
regionsPerServer[mra.toServer] = addRegion(regionsPerServer[mra.toServer], mra.region);
regionMoved(mra.region, mra.fromServer, mra.toServer);
break;
case SWAP_REGIONS:
assert action instanceof SwapRegionsAction: action.getClass();
SwapRegionsAction a = (SwapRegionsAction) action;
regionsPerServer[a.fromServer] = replaceRegion(regionsPerServer[a.fromServer], a.fromRegion, a.toRegion);
regionsPerServer[a.toServer] = replaceRegion(regionsPerServer[a.toServer], a.toRegion, a.fromRegion);
@ -1080,7 +1084,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
}
@Override
public void setClusterStatus(ClusterStatus st) {
public synchronized void setClusterStatus(ClusterStatus st) {
this.clusterStatus = st;
regionFinder.setClusterStatus(st);
}

View File

@ -94,6 +94,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
* so that the balancer gets the full picture of all loads on the cluster.</p>
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="Complaint is about costFunctions not being synchronized; not end of the world")
public class StochasticLoadBalancer extends BaseLoadBalancer {
protected static final String STEPS_PER_REGION_KEY =
@ -119,7 +121,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
private CandidateGenerator[] candidateGenerators;
private CostFromRegionLoadFunction[] regionLoadFunctions;
private CostFunction[] costFunctions;
private CostFunction[] costFunctions; // FindBugs: Wants this protected; IS2_INCONSISTENT_SYNC
// to save and report costs to JMX
private Double curOverallCost = 0d;

View File

@ -90,7 +90,7 @@ public class HFileLinkCleaner extends BaseHFileCleanerDelegate {
}
@Override
public void setConf(Configuration conf) {
public synchronized void setConf(Configuration conf) {
super.setConf(conf);
// setup filesystem

View File

@ -312,6 +312,8 @@ public class DisableTableProcedure
* Rollback of table state change in prepareDisable()
* @param env MasterProcedureEnv
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intended")
private void undoTableStateChange(final MasterProcedureEnv env) {
if (!skipTableStateCheck) {
try {
@ -322,6 +324,7 @@ public class DisableTableProcedure
}
} catch (Exception e) {
// Ignore exception.
LOG.trace(e.getMessage());
}
}
}

View File

@ -154,6 +154,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
* call should get implemented for each snapshot flavor.
*/
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intentional")
public void process() {
String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
+ eventType + " on table " + snapshotTable;
@ -205,7 +207,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
status.markComplete(msg);
LOG.info(msg);
metricsSnapshot.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
} catch (Exception e) {
} catch (Exception e) { // FindBugs: REC_CATCH_EXCEPTION
status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
snapshotTable + " because " + e.getMessage());
String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)

View File

@ -98,7 +98,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
* by this Handler.
* @return a string representing the method call without parameters
*/
public String getRPC() {
public synchronized String getRPC() {
return getRPC(false);
}
@ -166,7 +166,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
* @return true if the monitored handler is currently servicing an RPC call
* to a database command.
*/
public boolean isOperationRunning() {
public synchronized boolean isOperationRunning() {
if(!isRPCRunning()) {
return false;
}
@ -212,7 +212,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
}
@Override
public void markComplete(String status) {
public synchronized void markComplete(String status) {
super.markComplete(status);
this.params = null;
this.packet = null;

View File

@ -21,6 +21,9 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="FindBugs seems confused; says globalLimiter and lastUpdate " +
"are mostly synchronized...but to me it looks like they are totally synchronized")
public class QuotaState {
private long lastUpdate = 0;
private long lastQuery = 0;

View File

@ -38,6 +38,9 @@ import com.google.common.annotations.VisibleForTesting;
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="FindBugs seems confused; says limit and tlimit " +
"are mostly synchronized...but to me it looks like they are totally synchronized")
public abstract class RateLimiter {
public static final String QUOTA_RATE_LIMITER_CONF_KEY = "hbase.quota.rate.limiter";
private long tunit = 1000; // Timeunit factor for translating to ms.
@ -66,7 +69,7 @@ public abstract class RateLimiter {
* @param limit The max value available resource units can be refilled to.
* @param timeUnit Timeunit factor for translating to ms.
*/
public void set(final long limit, final TimeUnit timeUnit) {
public synchronized void set(final long limit, final TimeUnit timeUnit) {
switch (timeUnit) {
case MILLISECONDS:
tunit = 1;
@ -92,10 +95,11 @@ public abstract class RateLimiter {
public String toString() {
String rateLimiter = this.getClass().getSimpleName();
if (limit == Long.MAX_VALUE) {
if (getLimit() == Long.MAX_VALUE) {
return rateLimiter + "(Bypass)";
}
return rateLimiter + "(avail=" + avail + " limit=" + limit + " tunit=" + tunit + ")";
return rateLimiter + "(avail=" + getAvailable() + " limit=" + getLimit() +
" tunit=" + getTimeUnitInMillis() + ")";
}
/**
@ -113,7 +117,7 @@ public abstract class RateLimiter {
}
public synchronized boolean isBypass() {
return limit == Long.MAX_VALUE;
return getLimit() == Long.MAX_VALUE;
}
public synchronized long getLimit() {
@ -124,7 +128,7 @@ public abstract class RateLimiter {
return avail;
}
protected long getTimeUnitInMillis() {
protected synchronized long getTimeUnitInMillis() {
return tunit;
}
@ -188,7 +192,7 @@ public abstract class RateLimiter {
*/
public synchronized long waitInterval(final long amount) {
// TODO Handle over quota?
return (amount <= avail) ? 0 : getWaitInterval(limit, avail, amount);
return (amount <= avail) ? 0 : getWaitInterval(getLimit(), avail, amount);
}
// These two method are for strictly testing purpose only

View File

@ -27,6 +27,9 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="FindBugs seems confused; says bypassGlobals, namepaceLimiters, and " +
"tableLimiters are mostly synchronized...but to me it looks like they are totally synchronized")
public class UserQuotaState extends QuotaState {
private Map<String, QuotaLimiter> namespaceLimiters = null;
private Map<TableName, QuotaLimiter> tableLimiters = null;

View File

@ -389,7 +389,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Set when a flush has been requested.
volatile boolean flushRequested = false;
// Number of compactions running.
volatile int compacting = 0;
AtomicInteger compacting = new AtomicInteger(0);
// Gets set in close. If set, cannot compact or flush again.
volatile boolean writesEnabled = true;
// Set if region is read-only
@ -823,7 +823,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
this.writestate.flushRequested = false;
this.writestate.compacting = 0;
this.writestate.compacting.set(0);
if (this.writestate.writesEnabled) {
// Remove temporary data left over from old regions
@ -1367,6 +1367,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.closing.set(closing);
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
justification="I think FindBugs is confused")
private Map<byte[], List<StoreFile>> doClose(final boolean abort, MonitoredTask status)
throws IOException {
if (isClosed()) {
@ -1404,7 +1406,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
// block waiting for the lock for closing
lock.writeLock().lock();
lock.writeLock().lock(); // FindBugs: Complains UL_UNRELEASED_LOCK_EXCEPTION_PATH but seems fine
this.closing.set(true);
status.setStatus("Disabling writes for close");
try {
@ -1533,7 +1535,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
boolean interrupted = false;
try {
while (writestate.compacting > 0 || writestate.flushing) {
while (writestate.compacting.get() > 0 || writestate.flushing) {
LOG.debug("waiting for " + writestate.compacting + " compactions"
+ (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
try {
@ -1890,7 +1892,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
synchronized (writestate) {
if (writestate.writesEnabled) {
wasStateSet = true;
++writestate.compacting;
writestate.compacting.incrementAndGet();
} else {
String msg = "NOT compacting region " + this + ". Writes disabled.";
LOG.info(msg);
@ -1916,8 +1918,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} finally {
if (wasStateSet) {
synchronized (writestate) {
--writestate.compacting;
if (writestate.compacting <= 0) {
writestate.compacting.decrementAndGet();
if (writestate.compacting.get() <= 0) {
writestate.notifyAll();
}
}
@ -2160,6 +2162,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
justification="FindBugs seems confused about trxId")
protected PrepareFlushResult internalPrepareFlushCache(final WAL wal, final long myseqid,
final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
throws IOException {
@ -2391,6 +2395,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return false;
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
justification="Intentional; notify is about completed flush")
protected FlushResult internalFlushCacheAndCommit(
final WAL wal, MonitoredTask status, final PrepareFlushResult prepareResult,
final Collection<Store> storesToFlush)
@ -4460,6 +4466,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
@VisibleForTesting
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
justification="Intentional; post memstore flush")
void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
@ -4696,6 +4704,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return prepareFlushResult;
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
justification="Intentional; cleared the memstore")
void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
"RegionEvent marker from WAL ", regionEvent);
@ -4926,6 +4936,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return refreshStoreFiles(false);
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
justification="Notify is about post replay. Intentional")
protected boolean refreshStoreFiles(boolean force) throws IOException {
if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
return false; // if primary nothing to do
@ -7922,6 +7934,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
justification="Intentional")
public void startRegionOperation(Operation op) throws IOException {
switch (op) {
case GET: // read operations

View File

@ -1963,7 +1963,6 @@ public class HRegionServer extends HasThread implements
public boolean reportRegionStateTransition(final RegionStateTransitionContext context) {
TransitionCode code = context.getCode();
long openSeqNum = context.getOpenSeqNum();
long masterSystemTime = context.getMasterSystemTime();
HRegionInfo[] hris = context.getHris();
ReportRegionStateTransitionRequest.Builder builder =

View File

@ -189,7 +189,7 @@ implements HeapSize, Map<K,V> {
*
* @return currently available bytes
*/
public long getMemFree() {
public synchronized long getMemFree() {
return memFree;
}
@ -208,7 +208,7 @@ implements HeapSize, Map<K,V> {
* @return currently used memory in bytes
*/
public long getMemUsed() {
return (memTotal - memFree); // FindBugs IS2_INCONSISTENT_SYNC
return (memTotal - getMemFree()); // FindBugs IS2_INCONSISTENT_SYNC
}
/**
@ -227,7 +227,7 @@ implements HeapSize, Map<K,V> {
*
* @return number of misses
*/
public long getMissCount() {
public synchronized long getMissCount() {
return missCount; // FindBugs IS2_INCONSISTENT_SYNC
}
@ -239,7 +239,7 @@ implements HeapSize, Map<K,V> {
*/
public double getHitRatio() {
return (double)((double)hitCount/
((double)(hitCount+missCount)));
((double)(hitCount + getMissCount())));
}
/**
@ -269,7 +269,7 @@ implements HeapSize, Map<K,V> {
* @return memory usage of map in bytes
*/
public long heapSize() {
return (memTotal - memFree);
return (memTotal - getMemFree());
}
//--------------------------------------------------------------------------
@ -824,6 +824,8 @@ implements HeapSize, Map<K,V> {
*
* @return Set of entries in hash
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="Unused debugging function that reads only")
public Set<Entry<K,V>> entryTableSet() {
Set<Entry<K,V>> entrySet = new HashSet<Entry<K,V>>();
Entry [] table = entries; // FindBugs IS2_INCONSISTENT_SYNC

View File

@ -60,7 +60,7 @@ public class MemStoreChunkPool {
final static float POOL_INITIAL_SIZE_DEFAULT = 0.0f;
// Static reference to the MemStoreChunkPool
private static MemStoreChunkPool globalInstance;
private static MemStoreChunkPool GLOBAL_INSTANCE;
/** Boolean whether we have disabled the memstore chunk pool entirely. */
static boolean chunkPoolDisabled = false;
@ -179,12 +179,14 @@ public class MemStoreChunkPool {
* @param conf
* @return the global MemStoreChunkPool instance
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DC_DOUBLECHECK",
justification="Intentional")
static MemStoreChunkPool getPool(Configuration conf) {
if (globalInstance != null) return globalInstance;
if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
synchronized (MemStoreChunkPool.class) {
if (chunkPoolDisabled) return null;
if (globalInstance != null) return globalInstance;
if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
float poolSizePercentage = conf.getFloat(CHUNK_POOL_MAXSIZE_KEY, POOL_MAX_SIZE_DEFAULT);
if (poolSizePercentage <= 0) {
chunkPoolDisabled = true;
@ -210,8 +212,8 @@ public class MemStoreChunkPool {
int initialCount = (int) (initialCountPercentage * maxCount);
LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
+ ", max count " + maxCount + ", initial count " + initialCount);
globalInstance = new MemStoreChunkPool(conf, chunkSize, maxCount, initialCount);
return globalInstance;
GLOBAL_INSTANCE = new MemStoreChunkPool(conf, chunkSize, maxCount, initialCount);
return GLOBAL_INSTANCE;
}
}

View File

@ -277,7 +277,7 @@ public class RegionCoprocessorHost
continue;
}
int priority = matcher.group(3).trim().isEmpty() ?
Coprocessor.PRIORITY_USER : Integer.valueOf(matcher.group(3));
Coprocessor.PRIORITY_USER : Integer.parseInt(matcher.group(3));
String cfgSpec = null;
try {
cfgSpec = matcher.group(4);

View File

@ -278,6 +278,8 @@ public class RegionServerCoprocessorHost extends
private RegionServerServices regionServerServices;
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="BC_UNCONFIRMED_CAST",
justification="Intentional; FB has trouble detecting isAssignableFrom")
public RegionServerEnvironment(final Class<?> implClass,
final Coprocessor impl, final int priority, final int seq,
final Configuration conf, final RegionServerServices services) {
@ -285,7 +287,7 @@ public class RegionServerCoprocessorHost extends
this.regionServerServices = services;
for (Object itf : ClassUtils.getAllInterfaces(implClass)) {
Class<?> c = (Class<?>) itf;
if (SingletonCoprocessorService.class.isAssignableFrom(c)) {
if (SingletonCoprocessorService.class.isAssignableFrom(c)) {// FindBugs: BC_UNCONFIRMED_CAST
this.regionServerServices.registerService(
((SingletonCoprocessorService) impl).getService());
break;

View File

@ -570,6 +570,8 @@ public class StoreFile {
return sb.toString();
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG",
justification="Will not overflow")
public static class WriterBuilder {
private final Configuration conf;
private final CacheConfig cacheConf;
@ -582,7 +584,6 @@ public class StoreFile {
private Path filePath;
private InetSocketAddress[] favoredNodes;
private HFileContext fileContext;
private boolean shouldDropCacheBehind = false;
public WriterBuilder(Configuration conf, CacheConfig cacheConf,
FileSystem fs) {
@ -650,8 +651,8 @@ public class StoreFile {
return this;
}
public WriterBuilder withShouldDropCacheBehind(boolean shouldDropCacheBehind) {
this.shouldDropCacheBehind = shouldDropCacheBehind;
public WriterBuilder withShouldDropCacheBehind(boolean shouldDropCacheBehind/*NOT USED!!*/) {
// TODO: HAS NO EFFECT!!! FIX!!
return this;
}
/**
@ -757,9 +758,6 @@ public class StoreFile {
private Cell lastDeleteFamilyCell = null;
private long deleteFamilyCnt = 0;
/** Bytes per Checksum */
protected int bytesPerChecksum;
TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
/* isTimeRangeTrackerSet keeps track if the timeRange has already been set
* When flushing a memstore, we set TimeRange and use this variable to

View File

@ -91,6 +91,8 @@ public class TimeRangeTracker implements Writable {
* If required, update the current TimestampRange to include timestamp
* @param timestamp the timestamp value to include
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MT_CORRECTNESS",
justification="Intentional")
void includeTimestamp(final long timestamp) {
// Do test outside of synchronization block. Synchronization in here can be problematic
// when many threads writing one Store -- they can all pile up trying to add in here.

View File

@ -43,18 +43,20 @@ import org.apache.zookeeper.KeeperException.SessionExpiredException;
* target cluster is an HBase cluster.
*/
@InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MT_CORRECTNESS",
justification="Thinks zkw needs to be synchronized access but should be fine as is.")
public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
implements Abortable {
private static final Log LOG = LogFactory.getLog(HBaseReplicationEndpoint.class);
private ZooKeeperWatcher zkw = null;
private ZooKeeperWatcher zkw = null; // FindBugs: MT_CORRECTNESS
private List<ServerName> regionServers = new ArrayList<ServerName>(0);
private volatile long lastRegionServerUpdate;
private long lastRegionServerUpdate;
protected void disconnect() {
if (zkw != null){
if (zkw != null) {
zkw.close();
}
}
@ -181,7 +183,7 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
* Set the list of region servers for that peer
* @param regionServers list of addresses for the region servers
*/
public void setRegionServers(List<ServerName> regionServers) {
public synchronized void setRegionServers(List<ServerName> regionServers) {
this.regionServers = regionServers;
lastRegionServerUpdate = System.currentTimeMillis();
}

View File

@ -116,7 +116,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
* Skips the entries which has original seqId. Only entries persisted via distributed log replay
* have their original seq Id fields set.
*/
private class SkipReplayedEditsFilter extends BaseWALEntryFilter {
private static class SkipReplayedEditsFilter extends BaseWALEntryFilter {
@Override
public Entry filter(Entry entry) {
// if orig seq id is set, skip replaying the entry

View File

@ -279,7 +279,6 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
Configuration conf = env.getConfiguration();
fs = FileSystem.get(conf);
for(Pair<byte[], String> el: familyPaths) {
Path p = new Path(el.getSecond());
Path stageFamily = new Path(bulkToken, Bytes.toString(el.getFirst()));
if(!fs.exists(stageFamily)) {
fs.mkdirs(stageFamily);

View File

@ -24,6 +24,7 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -107,7 +108,7 @@ public class TableAuthManager implements Closeable {
private Configuration conf;
private ZKPermissionWatcher zkperms;
private volatile long mtime;
private AtomicLong mtime = new AtomicLong(0);
private TableAuthManager(ZooKeeperWatcher watcher, Configuration conf)
throws IOException {
@ -219,7 +220,7 @@ public class TableAuthManager implements Closeable {
}
}
globalCache = newCache;
mtime++;
mtime.incrementAndGet();
} catch (IOException e) {
// Never happens
LOG.error("Error occured while updating the global cache", e);
@ -247,7 +248,7 @@ public class TableAuthManager implements Closeable {
}
tableCache.put(table, newTablePerms);
mtime++;
mtime.incrementAndGet();
}
/**
@ -271,7 +272,7 @@ public class TableAuthManager implements Closeable {
}
nsCache.put(namespace, newTablePerms);
mtime++;
mtime.incrementAndGet();
}
private PermissionCache<TablePermission> getTablePermissions(TableName table) {
@ -335,12 +336,6 @@ public class TableAuthManager implements Closeable {
return false;
}
private boolean authorize(List<TablePermission> perms,
TableName table, byte[] family,
Permission.Action action) {
return authorize(perms, table, family, null, action);
}
private boolean authorize(List<TablePermission> perms,
TableName table, byte[] family,
byte[] qualifier, Permission.Action action) {
@ -740,7 +735,7 @@ public class TableAuthManager implements Closeable {
}
public long getMTime() {
return mtime;
return mtime.get();
}
private static Map<ZooKeeperWatcher,TableAuthManager> managerMap =

View File

@ -124,7 +124,7 @@ public class AuthenticationTokenSecretManager
}
@Override
protected byte[] createPassword(AuthenticationTokenIdentifier identifier) {
protected synchronized byte[] createPassword(AuthenticationTokenIdentifier identifier) {
long now = EnvironmentEdgeManager.currentTime();
AuthenticationKey secretKey = currentKey;
identifier.setKeyId(secretKey.getKeyId());
@ -229,7 +229,7 @@ public class AuthenticationTokenSecretManager
return true;
}
AuthenticationKey getCurrentKey() {
synchronized AuthenticationKey getCurrentKey() {
return currentKey;
}
@ -338,8 +338,11 @@ public class AuthenticationTokenSecretManager
// clear any expired
removeExpiredKeys();
if (lastKeyUpdate + keyUpdateInterval < now) {
long localLastKeyUpdate;
synchronized (this) {
localLastKeyUpdate = lastKeyUpdate;
}
if (localLastKeyUpdate + keyUpdateInterval < now) {
// roll a new master key
rollCurrentKey();
}

View File

@ -117,6 +117,8 @@ import com.google.protobuf.Service;
* visibility labels
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MT_CORRECTNESS",
justification="FIX visibilityLabelService; Make Synchronized!!!")
public class VisibilityController extends BaseMasterAndRegionObserver implements
VisibilityLabelsService.Interface, CoprocessorService {
@ -134,7 +136,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
private Map<InternalScanner,String> scannerOwners =
new MapMaker().weakKeys().makeMap();
private VisibilityLabelService visibilityLabelService;
private VisibilityLabelService visibilityLabelService; // FindBugs: MT_CORRECTNESS FIX!!!
/** if we are active, usually true, only not true if "hbase.security.authorization"
has been set to false in site configuration */
@ -272,8 +274,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
// Read the entire labels table and populate the zk
if (e.getEnvironment().getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
this.labelsRegion = true;
synchronized (this) {
this.accessControllerAvailable = CoprocessorHost.getLoadedCoprocessors()
.contains(AccessController.class.getName());
}
// Defer the init of VisibilityLabelService on labels region until it is in recovering state.
if (!e.getEnvironment().getRegion().isRecovering()) {
initVisibilityLabelService(e.getEnvironment());

View File

@ -270,7 +270,7 @@ public class ExportSnapshot extends Configured implements Tool {
InputStream in = openSourceFile(context, inputInfo);
int bandwidthMB = context.getConfiguration().getInt(CONF_BANDWIDTH_MB, 100);
if (Integer.MAX_VALUE != bandwidthMB) {
in = new ThrottledInputStream(new BufferedInputStream(in), bandwidthMB * 1024 * 1024);
in = new ThrottledInputStream(new BufferedInputStream(in), bandwidthMB * 1024L * 1024L);
}
try {
@ -634,7 +634,6 @@ public class ExportSnapshot extends Configured implements Tool {
@Override
public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
Configuration conf = context.getConfiguration();
String snapshotName = conf.get(CONF_SNAPSHOT_NAME);
Path snapshotDir = new Path(conf.get(CONF_SNAPSHOT_DIR));
FileSystem fs = FileSystem.get(snapshotDir.toUri(), conf);

View File

@ -263,6 +263,8 @@ public final class SnapshotInfo extends Configured implements Tool {
private SnapshotManifest snapshotManifest;
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intentional")
public int run(String[] args) throws IOException, InterruptedException {
final Configuration conf = getConf();
boolean listSnapshots = false;
@ -300,7 +302,7 @@ public final class SnapshotInfo extends Configured implements Tool {
printUsageAndExit();
}
} catch (Exception e) {
printUsageAndExit();
printUsageAndExit(); // FindBugs: REC_CATCH_EXCEPTION
}
}

View File

@ -54,21 +54,10 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
import com.google.common.collect.Ordering;
import com.google.common.collect.TreeMultimap;
import com.google.protobuf.ServiceException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -90,11 +79,11 @@ import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin;
@ -147,6 +136,15 @@ import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
import com.google.common.collect.Ordering;
import com.google.common.collect.TreeMultimap;
import com.google.protobuf.ServiceException;
/**
* HBaseFsck (hbck) is a tool for checking and repairing region consistency and
* table integrity problems in a corrupted HBase.
@ -264,7 +262,7 @@ public class HBaseFsck extends Configured implements Closeable {
private Path sidelineDir = null;
private boolean rerun = false; // if we tried to fix something, rerun hbck
private static boolean summary = false; // if we want to print less output
private static boolean SUMMARY = false; // if we want to print less output
private boolean checkMetaOnly = false;
private boolean checkRegionBoundaries = false;
private boolean ignorePreCheckPermission = false; // if pre-check permission
@ -3580,9 +3578,8 @@ public class HBaseFsck extends Configured implements Closeable {
this.metaEntry = metaEntry;
}
public int getReplicaId() {
if (metaEntry != null) return metaEntry.getReplicaId();
return deployedReplicaId;
public synchronized int getReplicaId() {
return metaEntry != null? metaEntry.getReplicaId(): deployedReplicaId;
}
public synchronized void addServer(HRegionInfo hri, ServerName server) {
@ -3887,7 +3884,7 @@ public class HBaseFsck extends Configured implements Closeable {
}
errorList.add(errorCode);
if (!summary) {
if (!getSUMMARY()) {
System.out.println("ERROR: " + message);
}
errorCount++;
@ -3929,7 +3926,7 @@ public class HBaseFsck extends Configured implements Closeable {
*/
@Override
public synchronized void report(String message) {
if (! summary) {
if (!getSUMMARY()) {
System.out.println("ERROR: " + message);
}
showProgress = 0;
@ -3955,11 +3952,15 @@ public class HBaseFsck extends Configured implements Closeable {
@Override
public synchronized void print(String message) {
if (!summary) {
if (!getSUMMARY()) {
System.out.println(message);
}
}
private synchronized static boolean getSUMMARY() {
return SUMMARY;
}
@Override
public boolean tableHasErrors(TableInfo table) {
return errorTables.contains(table);
@ -3981,7 +3982,7 @@ public class HBaseFsck extends Configured implements Closeable {
@Override
public synchronized void progress() {
if (showProgress++ == progressThreshold) {
if (!summary) {
if (!getSUMMARY()) {
System.out.print(".");
}
showProgress = 0;
@ -4211,8 +4212,8 @@ public class HBaseFsck extends Configured implements Closeable {
* Set summary mode.
* Print only summary of the tables and status (OK or INCONSISTENT)
*/
void setSummary() {
summary = true;
synchronized static void setSummary() {
SUMMARY = true;
}
/**

View File

@ -68,12 +68,17 @@ public class IdReadWriteLock {
/** For testing */
@VisibleForTesting
int purgeAndGetEntryPoolSize() {
System.gc();
gc();
Threads.sleep(200);
lockPool.purge();
return lockPool.size();
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DM_GC", justification="Intentional")
private void gc() {
System.gc();
}
@VisibleForTesting
public void waitForWaiters(long id, int numWaiters) throws InterruptedException {
for (ReentrantReadWriteLock readWriteLock;;) {

View File

@ -104,11 +104,8 @@ public class MetaUtils {
* @return HRegion for meta region
* @throws IOException e
*/
public HRegion getMetaRegion() throws IOException {
if (this.metaRegion == null) {
openMetaRegion();
}
return this.metaRegion;
public synchronized HRegion getMetaRegion() throws IOException {
return this.metaRegion == null? openMetaRegion(): this.metaRegion;
}
/**

View File

@ -46,6 +46,8 @@ import java.util.RandomAccess;
* <p>
* Iterators are read-only. They cannot be used to remove elements.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UG_SYNC_SET_UNSYNC_GET",
justification="TODO: synchronization in here needs review!!!")
public class SortedList<E> implements List<E>, RandomAccess {
private volatile List<E> list;
private final Comparator<? super E> comparator;
@ -80,7 +82,7 @@ public class SortedList<E> implements List<E>, RandomAccess {
* method to get a reference for iterating over using the RandomAccess
* pattern.
*/
public List<E> get() {
public List<E> get() { // FindBugs: UG_SYNC_SET_UNSYNC_GET complaint. Fix!!
return list;
}
@ -185,7 +187,7 @@ public class SortedList<E> implements List<E>, RandomAccess {
}
@Override
public E get(int index) {
public synchronized E get(int index) {
return list.get(index);
}

View File

@ -47,6 +47,8 @@ public class ZKDataMigrator extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(ZKDataMigrator.class);
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Intended")
public int run(String[] as) throws Exception {
Configuration conf = getConf();
ZooKeeperWatcher zkw = null;

View File

@ -2276,9 +2276,7 @@ public class WALSplitter {
} else {
((Put) m).add(cell);
}
if (m != null) {
m.setDurability(durability);
}
previousCell = cell;
}

View File

@ -338,7 +338,7 @@ public class TestSplitLogManager {
assertTrue(task == task2);
LOG.debug("task = " + task);
assertEquals(1L, tot_mgr_resubmit.get());
assertEquals(1, task.incarnation);
assertEquals(1, task.incarnation.get());
assertEquals(0, task.unforcedResubmits.get());
assertTrue(task.isOrphan());
assertTrue(task.isUnassigned());