Browse Source

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 16 years ago
parent
commit
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 
     then it should be refreshed upon recovery so that the excluded hosts are 
     lost and the maps are re-executed. (Amar Kamat via ddas)
     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
 Release 0.20.1 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   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.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 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.hdfs.DFSClient.DFSOutputStream;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
@@ -439,7 +440,11 @@ public class DistributedFileSystem extends FileSystem {
   public boolean reportChecksumFailure(Path f, 
   public boolean reportChecksumFailure(Path f, 
     FSDataInputStream in, long inPos, 
     FSDataInputStream in, long inPos, 
     FSDataInputStream sums, long sumsPos) {
     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];
     LocatedBlock lblocks[] = new LocatedBlock[2];
 
 
     // Find block in data stream.
     // 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
    * Returns true if the node is in the process of being decommissioned
    */
    */
   public boolean isDecommissionInProgress() {
   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.
    * Returns true if the node has been decommissioned.
    */
    */
   public boolean isDecommissioned() {
   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() {
   public void setDecommissioned() {
     adminState = AdminStates.DECOMMISSIONED;
     adminState = AdminStates.DECOMMISSIONED;
@@ -348,4 +342,17 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.hostName = Text.readString(in);
     this.hostName = Text.readString(in);
     setAdminState(WritableUtils.readEnum(in, AdminStates.class));
     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;
 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;
 import org.apache.hadoop.hdfs.protocol.Block;
 
 
@@ -32,11 +36,11 @@ class BlocksMap {
    * Internal class for block metadata.
    * Internal class for block metadata.
    */
    */
   static class BlockInfo extends Block {
   static class BlockInfo extends Block {
-    private INodeFile          inode;
+    private INodeFile inode;
 
 
     /**
     /**
      * This array contains triplets of references.
      * 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
      * triplets[3*i] is the reference to the DatanodeDescriptor
      * and triplets[3*i+1] and triplets[3*i+2] are references 
      * and triplets[3*i+1] and triplets[3*i+2] are references 
      * to the previous and the next blocks, respectively, in the 
      * 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 
      * If this block is the head of the list then return the next block as 
      * the new head.
      * the new head.
      * @return the new head of the list or null if the list becomes
      * @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) {
     BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
       if(head == null)
       if(head == null)
@@ -266,6 +270,18 @@ class BlocksMap {
       }
       }
       return true;
       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> {
   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 */
   /** A set of blocks to be invalidated by this datanode */
   private Set<Block> invalidateBlocks = new TreeSet<Block>();
   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 
    * in case of errors (e.g. datanode does not report if an error occurs 
    * while writing the block).
    * while writing the block).
    */
    */
@@ -159,7 +159,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param networkLocation location of the data node in network
    * @param networkLocation location of the data node in network
    * @param capacity capacity of the data node, including space used by non-dfs
    * @param capacity capacity of the data node, including space used by non-dfs
    * @param dfsUsed the used space by dfs datanode
    * @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
    * @param xceiverCount # of data transfers at the data node
    */
    */
   public DatanodeDescriptor(DatanodeID nodeID,
   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.
    * Add block to the head of the list of blocks belonging to the data-node.
    */
    */
   boolean addBlock(BlockInfo b) {
   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 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) {
   boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
     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> {
   static private class BlockIterator implements Iterator<Block> {
     private BlockInfo current;
     private BlockInfo current;
@@ -463,4 +463,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
       lastBlocksScheduledRollTime = now;
       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);
+  }
 }
 }