浏览代码

HDFS-4034. Remove redundant null checks. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1430585 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 12 年之前
父节点
当前提交
6d4a091567

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

@@ -465,6 +465,8 @@ Release 2.0.3-alpha - Unreleased
     
     
     HDFS-4033. Miscellaneous findbugs 2 fixes. (eli)
     HDFS-4033. Miscellaneous findbugs 2 fixes. (eli)
 
 
+    HDFS-4034. Remove redundant null checks. (eli)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

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

@@ -1593,8 +1593,7 @@ public class DFSClient implements java.io.Closeable {
     if (shouldEncryptData()) {
     if (shouldEncryptData()) {
       synchronized (this) {
       synchronized (this) {
         if (encryptionKey == null ||
         if (encryptionKey == null ||
-            (encryptionKey != null &&
-             encryptionKey.expiryDate < Time.now())) {
+            encryptionKey.expiryDate < Time.now()) {
           LOG.debug("Getting new encryption token from NN");
           LOG.debug("Getting new encryption token from NN");
           encryptionKey = namenode.getDataEncryptionKey();
           encryptionKey = namenode.getDataEncryptionKey();
         }
         }

+ 1 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java

@@ -53,11 +53,7 @@ public abstract class DataTransferProtoUtil {
 
 
   public static ChecksumProto toProto(DataChecksum checksum) {
   public static ChecksumProto toProto(DataChecksum checksum) {
     ChecksumTypeProto type = HdfsProtoUtil.toProto(checksum.getChecksumType());
     ChecksumTypeProto type = HdfsProtoUtil.toProto(checksum.getChecksumType());
-    if (type == null) {
-      throw new IllegalArgumentException(
-          "Can't convert checksum to protobuf: " + checksum);
-    }
-
+    // ChecksumType#valueOf never returns null
     return ChecksumProto.newBuilder()
     return ChecksumProto.newBuilder()
       .setBytesPerChecksum(checksum.getBytesPerChecksum())
       .setBytesPerChecksum(checksum.getBytesPerChecksum())
       .setType(type)
       .setType(type)

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

@@ -1904,10 +1904,11 @@ public class DataNode extends Configured
   }
   }
 
 
   /**
   /**
-   * Get namenode corresponding to a block pool
+   * Get the NameNode corresponding to the given block pool.
+   *
    * @param bpid Block pool Id
    * @param bpid Block pool Id
    * @return Namenode corresponding to the bpid
    * @return Namenode corresponding to the bpid
-   * @throws IOException
+   * @throws IOException if unable to get the corresponding NameNode
    */
    */
   public DatanodeProtocolClientSideTranslatorPB getActiveNamenodeForBP(String bpid)
   public DatanodeProtocolClientSideTranslatorPB getActiveNamenodeForBP(String bpid)
       throws IOException {
       throws IOException {
@@ -1931,11 +1932,6 @@ public class DataNode extends Configured
     final String bpid = block.getBlockPoolId();
     final String bpid = block.getBlockPoolId();
     DatanodeProtocolClientSideTranslatorPB nn =
     DatanodeProtocolClientSideTranslatorPB nn =
       getActiveNamenodeForBP(block.getBlockPoolId());
       getActiveNamenodeForBP(block.getBlockPoolId());
-    if (nn == null) {
-      throw new IOException(
-          "Unable to synchronize block " + rBlock + ", since this DN "
-          + " has not acknowledged any NN as active.");
-    }
     
     
     long recoveryId = rBlock.getNewGenerationStamp();
     long recoveryId = rBlock.getNewGenerationStamp();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -203,9 +203,6 @@ abstract public class ReplicaInfo extends Block implements Replica {
       throw new IOException("detachBlock:Block not found. " + this);
       throw new IOException("detachBlock:Block not found. " + this);
     }
     }
     File meta = getMetaFile();
     File meta = getMetaFile();
-    if (meta == null) {
-      throw new IOException("Meta file not found for block " + this);
-    }
 
 
     if (HardLink.getLinkCount(file) > numLinks) {
     if (HardLink.getLinkCount(file) > numLinks) {
       DataNode.LOG.info("CopyOnWrite for block " + this);
       DataNode.LOG.info("CopyOnWrite for block " + this);

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -140,10 +140,8 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
         fc.close();
         fc.close();
         fc = null;
         fc = null;
       }
       }
-      if (fp != null) {
-        fp.close();
-        fp = null;
-      }
+      fp.close();
+      fp = null;
     } finally {
     } finally {
       IOUtils.cleanup(FSNamesystem.LOG, fc, fp);
       IOUtils.cleanup(FSNamesystem.LOG, fc, fp);
       doubleBuf = null;
       doubleBuf = null;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeResourceChecker.java

@@ -164,7 +164,7 @@ public class NameNodeResourceChecker {
     
     
     CheckedVolume newVolume = new CheckedVolume(dir, required);
     CheckedVolume newVolume = new CheckedVolume(dir, required);
     CheckedVolume volume = volumes.get(newVolume.getVolume());
     CheckedVolume volume = volumes.get(newVolume.getVolume());
-    if (volume == null || (volume != null && !volume.isRequired())) {
+    if (volume == null || !volume.isRequired()) {
       volumes.put(newVolume.getVolume(), newVolume);
       volumes.put(newVolume.getVolume(), newVolume);
     }
     }
   }
   }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -605,7 +605,9 @@ public class SecondaryNameNode implements Runnable {
       terminate(ret);
       terminate(ret);
     }
     }
 
 
-    secondary.startCheckpointThread();
+    if (secondary != null) {
+      secondary.startCheckpointThread();
+    }
   }
   }