HBASE-15115 Fix findbugs complaints in hbase-client

This commit is contained in:
stack 2016-01-15 10:25:49 -08:00
parent 032f5574b0
commit d9fd87d5a8
8 changed files with 26 additions and 19 deletions

View File

@ -103,7 +103,7 @@ public class MetricsConnection {
}
@VisibleForTesting
protected final class CallTracker {
protected static final class CallTracker {
private final String name;
@VisibleForTesting final Timer callTimer;
@VisibleForTesting final Histogram reqHist;

View File

@ -85,8 +85,7 @@ public class RetriesExhaustedException extends IOException {
public RetriesExhaustedException(final int numTries,
final List<ThrowableWithExtraContext> exceptions) {
super(getMessage(numTries, exceptions),
(exceptions != null && !exceptions.isEmpty() ?
exceptions.get(exceptions.size() - 1).t : null));
exceptions.isEmpty()? null: exceptions.get(exceptions.size() - 1).t);
}
private static String getMessage(String callableVitals, int numTries,

View File

@ -59,6 +59,8 @@ class RetryingCallerInterceptorFactory {
* {@link RetryingCallerInterceptor} object according to the
* configuration.
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
justification="Convert thrown exception to unchecked")
public RetryingCallerInterceptor build() {
RetryingCallerInterceptor ret = NO_OP_INTERCEPTOR;
if (failFast) {

View File

@ -186,12 +186,8 @@ public class FuzzyRowFilter extends FilterBase {
@Override
public int compare(Pair<byte[], Pair<byte[], byte[]>> o1,
Pair<byte[], Pair<byte[], byte[]>> o2) {
int compare = Bytes.compareTo(o1.getFirst(), o2.getFirst());
if (!isReversed()) {
return compare;
} else {
return -compare;
}
return isReversed()? Bytes.compareTo(o2.getFirst(), o1.getFirst()):
Bytes.compareTo(o1.getFirst(), o2.getFirst());
}
});
}

View File

@ -80,8 +80,6 @@ public class LongComparator extends ByteArrayComparable {
*/
boolean areSerializedFieldsEqual(LongComparator other) {
if (other == this) return true;
if (!(other instanceof LongComparator)) return false;
return super.areSerializedFieldsEqual(other);
}
}

View File

@ -499,6 +499,8 @@ public class MultiRowRangeFilter extends FilterBase {
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
justification="This compareTo is not of this Object, but of referenced RowRange")
public int compareTo(RowRange other) {
return Bytes.compareTo(this.startRow, other.startRow);
}

View File

@ -383,7 +383,7 @@ public class RpcClientImpl extends AbstractRpcClient {
}
}
private UserInformation getUserInfo(UserGroupInformation ugi) {
private synchronized UserInformation getUserInfo(UserGroupInformation ugi) {
if (ugi == null || authMethod == AuthMethod.DIGEST) {
// Don't send user for token auth
return null;
@ -804,7 +804,9 @@ public class RpcClientImpl extends AbstractRpcClient {
byte [] preamble = new byte [rpcHeaderLen + 2];
System.arraycopy(HConstants.RPC_HEADER, 0, preamble, 0, rpcHeaderLen);
preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
preamble[rpcHeaderLen + 1] = authMethod.code;
synchronized (this) {
preamble[rpcHeaderLen + 1] = authMethod.code;
}
outStream.write(preamble);
outStream.flush();
}
@ -880,6 +882,8 @@ public class RpcClientImpl extends AbstractRpcClient {
* threads.
* @see #readResponse()
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="Findbugs is misinterpreting locking missing fact that this.outLock is held")
private void writeRequest(Call call, final int priority, Span span) throws IOException {
RequestHeader.Builder builder = RequestHeader.newBuilder();
builder.setCallId(call.id);
@ -913,8 +917,8 @@ public class RpcClientImpl extends AbstractRpcClient {
checkIsOpen(); // Now we're checking that it didn't became idle in between.
try {
call.callStats.setRequestSizeBytes(
IPCUtil.write(this.out, header, call.param, cellBlock));
call.callStats.setRequestSizeBytes(IPCUtil.write(this.out, header, call.param,
cellBlock));
} catch (IOException e) {
// We set the value inside the synchronized block, this way the next in line
// won't even try to write. Otherwise we might miss a call in the calls map?
@ -932,14 +936,20 @@ public class RpcClientImpl extends AbstractRpcClient {
// We added a call, and may be started the connection close. In both cases, we
// need to notify the reader.
synchronized (this) {
notifyAll();
}
doNotify();
// Now that we notified, we can rethrow the exception if any. Otherwise we're good.
if (writeException != null) throw writeException;
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
justification="Presume notifyAll is because we are closing/shutting down")
private synchronized void doNotify() {
// Make a separate method so can do synchronize and add findbugs annotation; only one
// annotation at at time in source 1.7.
notifyAll(); // Findbugs: NN_NAKED_NOTIFY
}
/* Receive a response.
* Because only one receiver, so no synchronization on in.
*/

View File

@ -495,7 +495,7 @@ public final class RequestConverter {
builder.setScan(ProtobufUtil.toScan(scan));
builder.setClientHandlesPartials(true);
builder.setClientHandlesHeartbeats(true);
builder.setTrackScanMetrics(scan != null && scan.isScanMetricsEnabled());
builder.setTrackScanMetrics(scan.isScanMetricsEnabled());
return builder.build();
}