HDFS-3039. Address findbugs and javadoc warnings on branch. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1296017 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2012-03-02 01:32:49 +00:00
parent 55832c6c96
commit 7be4e5bd22
15 changed files with 55 additions and 39 deletions

View File

@ -282,4 +282,8 @@
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ipc\.protobuf\.IpcConnectionContextProtos.*"/>
</Match>
<Match>
<!-- protobuf generated code -->
<Class name="~org\.apache\.hadoop\.ha\.proto\.HAServiceProtocolProtos.*"/>
</Match>
</FindBugsFilter>

View File

@ -234,7 +234,7 @@ public synchronized void quitElection() {
/**
* Exception thrown when there is no active leader
*/
public class ActiveNotFoundException extends Exception {
public static class ActiveNotFoundException extends Exception {
private static final long serialVersionUID = 3505396722342846462L;
}

View File

@ -262,8 +262,7 @@ protected int runCmd(String[] argv) throws Exception {
return -1;
}
int i = 0;
String cmd = argv[i++];
String cmd = argv[0];
if (!cmd.startsWith("-")) {
errOut.println("Bad command '" + cmd + "': expected command starting with '-'");

View File

@ -76,7 +76,7 @@ public void checkArgs(String argStr) throws BadFencingConfigurationException {
if (argStr != null) {
// Use a dummy service when checking the arguments defined
// in the configuration are parseable.
Args args = new Args(new InetSocketAddress("localhost", 8020), argStr);
new Args(new InetSocketAddress("localhost", 8020), argStr);
}
}

View File

@ -30,7 +30,7 @@ public class ThreadUtil {
/**
* Cause the current thread to sleep as close as possible to the provided
* number of milliseconds. This method will log and ignore any
* {@link InterrupedException} encountered.
* {@link InterruptedException} encountered.
*
* @param millis the number of milliseconds for the current thread to sleep
*/

View File

@ -248,3 +248,5 @@ HDFS-3023. Optimize entries in edits log for persistBlocks call. (todd)
HDFS-2979. Balancer should use logical uri for creating failover proxy with HA enabled. (atm)
HDFS-3035. Fix failure of TestFileAppendRestart due to OP_UPDATE_BLOCKS (todd)
HDFS-3039. Address findbugs and javadoc warnings on branch. (todd via atm)

View File

@ -247,4 +247,12 @@
<Method name="save" />
<Bug pattern="OS_OPEN_STREAM" />
</Match>
<!--
This method isn't performance-critical and is much clearer to write as it's written.
-->
<Match>
<Class name="org.apache.hadoop.hdfs.server.datanode.BlockPoolManager" />
<Method name="doRefreshNamenodes" />
<Bug category="PERFORMANCE" />
</Match>
</FindBugsFilter>

View File

@ -99,7 +99,8 @@ public static String getNameNodeId(Configuration conf, String nsId) {
nsId, null, DFSUtil.LOCAL_ADDRESS_MATCHER);
if (suffixes == null) {
String msg = "Configuration " + DFS_NAMENODE_RPC_ADDRESS_KEY +
" must be suffixed with" + namenodeId + " for HA configuration.";
" must be suffixed with nameservice and namenode ID for HA " +
"configuration.";
throw new HadoopIllegalArgumentException(msg);
}

View File

@ -63,7 +63,8 @@ public class NameNodeProxies {
/**
* Wrapper for a client proxy as well as its associated service ID.
* This is simply used as a tuple-like return type for
* {@link createProxy} and {@link createNonHaProxy}.
* {@link NameNodeProxies#createProxy} and
* {@link NameNodeProxies#createNonHAProxy}.
*/
public static class ProxyAndInfo<PROXYTYPE> {
private final PROXYTYPE proxy;
@ -125,7 +126,7 @@ public static <T> ProxyAndInfo<T> createProxy(Configuration conf,
/**
* Creates an explicitly non-HA-enabled proxy object. Most of the time you
* don't want to use this, and should instead use {@link createProxy}.
* don't want to use this, and should instead use {@link NameNodeProxies#createProxy}.
*
* @param conf the configuration object
* @param nnAddr address of the remote NN to connect to
@ -160,8 +161,8 @@ public static <T> ProxyAndInfo<T> createNonHAProxy(
conf, ugi);
} else {
String message = "Upsupported protocol found when creating the proxy " +
"conection to NameNode: " +
((xface != null) ? xface.getClass().getName() : xface);
"connection to NameNode: " +
((xface != null) ? xface.getClass().getName() : "null");
LOG.error(message);
throw new IllegalStateException(message);
}

View File

@ -1918,7 +1918,7 @@ private void addStoredBlockImmediate(BlockInfo storedBlock,
int numCurrentReplica = countLiveNodes(storedBlock);
if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
&& numCurrentReplica >= minReplication) {
storedBlock = completeBlock(storedBlock.getINode(), storedBlock, false);
completeBlock(storedBlock.getINode(), storedBlock, false);
} else if (storedBlock.isComplete()) {
// check whether safe replication is reached for the block
// only complete blocks are counted towards that.

View File

@ -173,7 +173,7 @@ String getBlockPoolId() {
}
}
NamespaceInfo getNamespaceInfo() {
synchronized NamespaceInfo getNamespaceInfo() {
return bpNSInfo;
}
@ -366,7 +366,7 @@ private static void checkNSEquality(
}
}
DatanodeRegistration createRegistration() {
synchronized DatanodeRegistration createRegistration() {
Preconditions.checkState(bpNSInfo != null,
"getRegistration() can only be called after initial handshake");
return dn.createBPRegistration(bpNSInfo);

View File

@ -188,7 +188,7 @@ private void init(Configuration conf, NNStorage storage, List<URI> editsDirs) {
this.sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf);
}
public void initJournalsForWrite() {
public synchronized void initJournalsForWrite() {
Preconditions.checkState(state == State.UNINITIALIZED ||
state == State.CLOSED, "Unexpected state: %s", state);
@ -196,7 +196,7 @@ public void initJournalsForWrite() {
state = State.BETWEEN_LOG_SEGMENTS;
}
public void initSharedJournalsForRead() {
public synchronized void initSharedJournalsForRead() {
if (state == State.OPEN_FOR_READING) {
LOG.warn("Initializing shared journals for READ, already open for READ",
new Exception());
@ -209,7 +209,7 @@ public void initSharedJournalsForRead() {
state = State.OPEN_FOR_READING;
}
private void initJournals(List<URI> dirs) {
private synchronized void initJournals(List<URI> dirs) {
int minimumRedundantJournals = conf.getInt(
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY,
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT);
@ -808,7 +808,7 @@ void logReassignLease(String leaseHolder, String src, String newHolder) {
* Used only by unit tests.
*/
@VisibleForTesting
List<JournalAndStream> getJournals() {
synchronized List<JournalAndStream> getJournals() {
return journalSet.getAllJournalStreams();
}
@ -816,7 +816,7 @@ List<JournalAndStream> getJournals() {
* Used only by tests.
*/
@VisibleForTesting
public JournalSet getJournalSet() {
synchronized public JournalSet getJournalSet() {
return journalSet;
}
@ -950,17 +950,14 @@ synchronized void abortCurrentLogSegment() {
/**
* Archive any log files that are older than the given txid.
*/
public void purgeLogsOlderThan(final long minTxIdToKeep) {
synchronized (this) {
// synchronized to prevent findbugs warning about inconsistent
// synchronization. This will be JIT-ed out if asserts are
// off.
public synchronized void purgeLogsOlderThan(final long minTxIdToKeep) {
assert curSegmentTxId == HdfsConstants.INVALID_TXID || // on format this is no-op
minTxIdToKeep <= curSegmentTxId :
"cannot purge logs older than txid " + minTxIdToKeep +
" when current segment starts at " + curSegmentTxId;
}
// This could be improved to not need synchronization. But currently,
// journalSet is not threadsafe, so we need to synchronize this method.
try {
journalSet.purgeLogsOlderThan(minTxIdToKeep);
} catch (IOException ex) {
@ -992,7 +989,7 @@ synchronized long getSyncTxId() {
// sets the initial capacity of the flush buffer.
public void setOutputBufferCapacity(int size) {
synchronized void setOutputBufferCapacity(int size) {
journalSet.setOutputBufferCapacity(size);
}
@ -1069,7 +1066,7 @@ synchronized void logEdit(final int length, final byte[] data) {
/**
* Run recovery on all journals to recover any unclosed segments
*/
void recoverUnclosedStreams() {
synchronized void recoverUnclosedStreams() {
Preconditions.checkState(
state == State.BETWEEN_LOG_SEGMENTS,
"May not recover segments - wrong state: %s", state);
@ -1092,7 +1089,7 @@ Collection<EditLogInputStream> selectInputStreams(long fromTxId,
* @param toAtLeast the selected streams must contain this transaction
* @param inProgessOk set to true if in-progress streams are OK
*/
public Collection<EditLogInputStream> selectInputStreams(long fromTxId,
public synchronized Collection<EditLogInputStream> selectInputStreams(long fromTxId,
long toAtLeastTxId, boolean inProgressOk) throws IOException {
List<EditLogInputStream> streams = new ArrayList<EditLogInputStream>();
EditLogInputStream stream = journalSet.getInputStream(fromTxId, inProgressOk);

View File

@ -494,7 +494,7 @@ void startCommonServices(Configuration conf, HAContext haContext) throws IOExcep
nnResourceChecker = new NameNodeResourceChecker(conf);
checkAvailableResources();
assert safeMode != null &&
!safeMode.initializedReplQueues;
!safeMode.isPopulatingReplQueues();
setBlockTotal();
blockManager.activate(conf);
this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
@ -3801,7 +3801,7 @@ private void doConsistencyCheck() {
}
}
private void adjustBlockTotals(int deltaSafe, int deltaTotal) {
private synchronized void adjustBlockTotals(int deltaSafe, int deltaTotal) {
if (!shouldIncrementallyTrackBlocks) {
return;
}

View File

@ -310,7 +310,9 @@ synchronized public void recoverUnfinalizedSegments() throws IOException {
// file, but before writing anything to it. Safe to delete it.
if (elf.getFile().length() == 0) {
LOG.info("Deleting zero-length edit log file " + elf);
elf.getFile().delete();
if (!elf.getFile().delete()) {
throw new IOException("Unable to delete file " + elf.getFile());
}
continue;
}
@ -328,7 +330,9 @@ synchronized public void recoverUnfinalizedSegments() throws IOException {
// delete the file.
if (elf.getNumTransactions() == 0) {
LOG.info("Deleting edit log file with zero transactions " + elf);
elf.getFile().delete();
if (!elf.getFile().delete()) {
throw new IOException("Unable to delete " + elf.getFile());
}
continue;
}

View File

@ -315,10 +315,10 @@ void generateHealthReport(JspWriter out, NameNode nn,
// since the standby namenode doesn't compute replication queues
String underReplicatedBlocks = "";
if (nn.getServiceState() == HAServiceState.ACTIVE) {
underReplicatedBlocks = new String(rowTxt()
underReplicatedBlocks = rowTxt()
+ colTxt("Excludes missing blocks.")
+ "Number of Under-Replicated Blocks" + colTxt() + ":" + colTxt()
+ fsn.getBlockManager().getUnderReplicatedNotMissingBlocks());
+ fsn.getBlockManager().getUnderReplicatedNotMissingBlocks();
}
out.print("<div id=\"dfstable\"> <table>\n" + rowTxt() + colTxt()
+ "Configured Capacity" + colTxt() + ":" + colTxt()