Explorar o código

Merge HDFS-2564 (Cleanup unnecessary exceptions thrown and unnecessary casts)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1309487 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins %!s(int64=13) %!d(string=hai) anos
pai
achega
ed22c28058

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -206,6 +206,9 @@ Release 2.0.0 - UNRELEASED
     HDFS-3168. Remove unnecessary "throw IOException" and change fields to
     final in FSNamesystem and BlockManager.  (szetszwo)
 
+    HDFS-2564. Cleanup unnecessary exceptions thrown and unnecessary casts.
+    (Hari Mankude via eli)
+
   OPTIMIZATIONS
 
     HDFS-2477. Optimize computing the diff between a block report and the

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -541,7 +541,7 @@ public class DFSInputStream extends FSInputStream {
           if (pos > blockEnd) {
             currentNode = blockSeekTo(pos);
           }
-          int realLen = (int) Math.min((long) len, (blockEnd - pos + 1L));
+          int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
           int result = readBuffer(buf, off, realLen, corruptedBlockMap);
           
           if (result >= 0) {

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -229,8 +229,7 @@ public class DataNode extends Configured
    * Use {@link NetUtils#createSocketAddr(String)} instead.
    */
   @Deprecated
-  public static InetSocketAddress createSocketAddr(String target
-                                                   ) throws IOException {
+  public static InetSocketAddress createSocketAddr(String target) {
     return NetUtils.createSocketAddr(target);
   }
   
@@ -1350,7 +1349,7 @@ public class DataNode extends Configured
      * entire target list, the block, and the data.
      */
     DataTransfer(DatanodeInfo targets[], ExtendedBlock b, BlockConstructionStage stage,
-        final String clientname) throws IOException {
+        final String clientname)  {
       if (DataTransferProtocol.LOG.isDebugEnabled()) {
         DataTransferProtocol.LOG.debug(getClass().getSimpleName() + ": "
             + b + " (numBytes=" + b.getNumBytes() + ")"

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -244,7 +244,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * Return total capacity, used and unused
    */
   @Override // FSDatasetMBean
-  public long getCapacity() throws IOException {
+  public long getCapacity() {
     synchronized(statsLock) {
       return volumes.getCapacity();
     }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java

@@ -56,9 +56,8 @@ public interface FSDatasetMBean {
   /**
    * Returns total capacity (in bytes) of storage (used and unused)
    * @return  total capacity of storage (used and unused)
-   * @throws IOException
    */
-  public long getCapacity() throws IOException;
+  public long getCapacity();
 
   /**
    * Returns the amount of free storage space (in bytes)

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1798,7 +1798,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       lb = startFileInternal(src, null, holder, clientMachine, 
                         EnumSet.of(CreateFlag.APPEND), 
-                        false, blockManager.maxReplication, (long)0);
+                        false, blockManager.maxReplication, 0);
     } finally {
       writeUnlock();
     }
@@ -1883,7 +1883,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       fileLength = pendingFile.computeContentSummary().getLength();
       blockSize = pendingFile.getPreferredBlockSize();
       clientNode = pendingFile.getClientNode();
-      replication = (int)pendingFile.getReplication();
+      replication = pendingFile.getReplication();
     } finally {
       writeUnlock();
     }
@@ -2657,6 +2657,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     // If the penultimate block is not COMPLETE, then it must be COMMITTED.
     if(nrCompleteBlocks < nrBlocks - 2 ||
        nrCompleteBlocks == nrBlocks - 2 &&
+         curBlock != null &&
          curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       final String message = "DIR* NameSystem.internalReleaseLease: "
         + "attempt to release a create lock on "
@@ -2742,7 +2743,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   
   Lease reassignLeaseInternal(Lease lease, String src, String newHolder,
-      INodeFileUnderConstruction pendingFile) throws IOException {
+      INodeFileUnderConstruction pendingFile) {
     assert hasWriteLock();
     pendingFile.setClientName(newHolder);
     return leaseManager.reassignLease(lease, src, newHolder);
@@ -2850,7 +2851,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                 newtargets[i]);
           }
         }
-        if (closeFile) {
+        if ((closeFile) && (descriptors != null)) {
           // the file is getting closed. Insert block locations into blockManager.
           // Otherwise fsck will report these blocks as MISSING, especially if the
           // blocksReceived from Datanodes take a long time to arrive.
@@ -3592,7 +3593,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.blockTotal = total;
       this.blockThreshold = (int) (blockTotal * threshold);
       this.blockReplQueueThreshold = 
-        (int) (((double) blockTotal) * replQueueThreshold);
+        (int) (blockTotal * replQueueThreshold);
       if (haEnabled) {
         // After we initialize the block count, any further namespace
         // modifications done while in safe mode need to keep track