Explorar o código

HADOOP-5841. Resolve findbugs warnings in DistributedFileSystem, DatanodeInfo, BlocksMap, DataNodeDescriptor. Contributed by Jakob Homan

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@776794 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze %!s(int64=16) %!d(string=hai) anos
pai
achega
11f3630562

+ 3 - 0
CHANGES.txt

@@ -640,6 +640,9 @@ Trunk (unreleased changes)
     then it should be refreshed upon recovery so that the excluded hosts are 
     lost and the maps are re-executed. (Amar Kamat via ddas)
 
+    HADOOP-5841. Resolve findbugs warnings in DistributedFileSystem,
+    DatanodeInfo, BlocksMap, DataNodeDescriptor.  (Jakob Homan via szetszwo)
+
 Release 0.20.1 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 6 - 1
src/hdfs/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Progressable;
@@ -439,7 +440,11 @@ public class DistributedFileSystem extends FileSystem {
   public boolean reportChecksumFailure(Path f, 
     FSDataInputStream in, long inPos, 
     FSDataInputStream sums, long sumsPos) {
-
+    
+    if(!(in instanceof DFSDataInputStream && sums instanceof DFSDataInputStream))
+      throw new IllegalArgumentException("Input streams must be types " +
+                                         "of DFSDataInputStream");
+    
     LocatedBlock lblocks[] = new LocatedBlock[2];
 
     // Find block in data stream.

+ 16 - 9
src/hdfs/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -246,24 +246,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * Returns true if the node is in the process of being decommissioned
    */
   public boolean isDecommissionInProgress() {
-    if (adminState == AdminStates.DECOMMISSION_INPROGRESS) {
-      return true;
-    }
-    return false;
+    return adminState == AdminStates.DECOMMISSION_INPROGRESS;
   }
 
   /**
    * Returns true if the node has been decommissioned.
    */
   public boolean isDecommissioned() {
-    if (adminState == AdminStates.DECOMMISSIONED) {
-      return true;
-    }
-    return false;
+    return adminState == AdminStates.DECOMMISSIONED;
   }
 
   /**
-   * Sets the admin state to indicate that decommision is complete.
+   * Sets the admin state to indicate that decommission is complete.
    */
   public void setDecommissioned() {
     adminState = AdminStates.DECOMMISSIONED;
@@ -348,4 +342,17 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.hostName = Text.readString(in);
     setAdminState(WritableUtils.readEnum(in, AdminStates.class));
   }
+
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to use super equality as datanodes are uniquely identified
+    // by DatanodeID
+    return (this == obj) || super.equals(obj);
+  }
 }

+ 20 - 4
src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -17,7 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 
@@ -32,11 +36,11 @@ class BlocksMap {
    * Internal class for block metadata.
    */
   static class BlockInfo extends Block {
-    private INodeFile          inode;
+    private INodeFile inode;
 
     /**
      * This array contains triplets of references.
-     * For each i-th data-node the block belongs to
+     * For each i-th datanode the block belongs to
      * triplets[3*i] is the reference to the DatanodeDescriptor
      * and triplets[3*i+1] and triplets[3*i+2] are references 
      * to the previous and the next blocks, respectively, in the 
@@ -217,7 +221,7 @@ class BlocksMap {
      * If this block is the head of the list then return the next block as 
      * the new head.
      * @return the new head of the list or null if the list becomes
-     * empy after deletion.
+     * empty after deletion.
      */
     BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
       if(head == null)
@@ -266,6 +270,18 @@ class BlocksMap {
       }
       return true;
     }
+
+    @Override
+    public int hashCode() {
+      // Super implementation is sufficient
+      return super.hashCode();
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+      // Sufficient to rely on super's implementation
+      return (this == obj) || super.equals(obj);
+    }
   }
 
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {

+ 19 - 6
src/hdfs/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -94,8 +94,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A set of blocks to be invalidated by this datanode */
   private Set<Block> invalidateBlocks = new TreeSet<Block>();
 
-  /* Variables for maintaning number of blocks scheduled to be written to
-   * this datanode. This count is approximate and might be slightly higger
+  /* Variables for maintaining number of blocks scheduled to be written to
+   * this datanode. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs 
    * while writing the block).
    */
@@ -159,7 +159,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param networkLocation location of the data node in network
    * @param capacity capacity of the data node, including space used by non-dfs
    * @param dfsUsed the used space by dfs datanode
-   * @param remaining remaing capacity of the data node
+   * @param remaining remaining capacity of the data node
    * @param xceiverCount # of data transfers at the data node
    */
   public DatanodeDescriptor(DatanodeID nodeID,
@@ -174,7 +174,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * Add data-node to the block.
+   * Add datanode to the block.
    * Add block to the head of the list of blocks belonging to the data-node.
    */
   boolean addBlock(BlockInfo b) {
@@ -187,7 +187,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   
   /**
    * Remove block from the list of blocks belonging to the data-node.
-   * Remove data-node from the block.
+   * Remove datanode from the block.
    */
   boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
@@ -228,7 +228,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * Iterates over the list of blocks belonging to the data-node.
+   * Iterates over the list of blocks belonging to the datanode.
    */
   static private class BlockIterator implements Iterator<Block> {
     private BlockInfo current;
@@ -463,4 +463,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
       lastBlocksScheduledRollTime = now;
     }
   }
+  
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to use super equality as datanodes are uniquely identified
+    // by DatanodeID
+    return (this == obj) || super.equals(obj);
+  }
 }