瀏覽代碼

HDFS-6276. Merge r1589586 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1589588 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 年之前
父節點
當前提交
32e76decb5

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

@@ -130,6 +130,8 @@ Release 2.5.0 - UNRELEASED
     HDFS-6217. Webhdfs PUT operations may not work via a http proxy.
     (Daryn Sharp via kihwal)
 
+    HDFS-6276. Remove unnecessary conditions and null check. (suresh)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -423,10 +423,8 @@ public class BlockManager {
 
   public void close() {
     try {
-      if (replicationThread != null) {
-        replicationThread.interrupt();
-        replicationThread.join(3000);
-      }
+      replicationThread.interrupt();
+      replicationThread.join(3000);
     } catch (InterruptedException ie) {
     }
     datanodeManager.close();
@@ -818,7 +816,7 @@ public class BlockManager {
       for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
         final DatanodeDescriptor d = storage.getDatanodeDescriptor();
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
-        if (isCorrupt || (!isCorrupt && !replicaCorrupt))
+        if (isCorrupt || (!replicaCorrupt))
           machines[j++] = storage;
       }
     }
@@ -2245,7 +2243,6 @@ public class BlockManager {
       // it will happen in next block report otherwise.
       return block;
     }
-    assert storedBlock != null : "Block must be stored by now";
     BlockCollection bc = storedBlock.getBlockCollection();
     assert bc != null : "Block must belong to a file";
 

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -86,8 +86,7 @@ class BlockReceiver implements Closeable {
   private int bytesPerChecksum;
   private int checksumSize;
   
-  private final PacketReceiver packetReceiver =
-      new PacketReceiver(false);
+  private final PacketReceiver packetReceiver = new PacketReceiver(false);
   
   protected final String inAddr;
   protected final String myAddr;
@@ -268,10 +267,8 @@ class BlockReceiver implements Closeable {
    */
   @Override
   public void close() throws IOException {
-    if (packetReceiver != null) {
-      packetReceiver.close();
-    }
-    
+    packetReceiver.close();
+
     IOException ioe = null;
     if (syncOnClose && (out != null || checksumOut != null)) {
       datanode.metrics.incrFsyncCount();      

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

@@ -131,8 +131,7 @@ public class DataBlockScanner implements Runnable {
   private BlockPoolSliceScanner getNextBPScanner(String currentBpId) {
     
     String nextBpId = null;
-    while ((nextBpId == null) && datanode.shouldRun
-        && !blockScannerThread.isInterrupted()) {
+    while (datanode.shouldRun && !blockScannerThread.isInterrupted()) {
       waitForInit();
       synchronized (this) {
         if (getBlockPoolSetSize() > 0) {          

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RollingLogsImpl.java

@@ -188,11 +188,9 @@ class RollingLogsImpl implements RollingLogs {
         if (reader != null && (line = reader.readLine()) != null) {
           return;
         }
-        if (line == null) {
-          // move to the next file.
-          if (openFile()) {
-            readNext();
-          }
+        // move to the next file.
+        if (openFile()) {
+          readNext();
         }
       } finally {
         if (!hasNext()) {

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

@@ -109,9 +109,7 @@ public final class CachePool {
     UserGroupInformation ugi = null;
     String ownerName = info.getOwnerName();
     if (ownerName == null) {
-      if (ugi == null) {
-        ugi = NameNode.getRemoteUser();
-      }
+      ugi = NameNode.getRemoteUser();
       ownerName = ugi.getShortUserName();
     }
     String groupName = info.getGroupName();

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

@@ -655,7 +655,7 @@ public class FSImageFormat {
             clientName = FSImageSerialization.readString(in);
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
-            if (blocks != null && blocks.length > 0) {
+            if (blocks.length > 0) {
               BlockInfo lastBlk = blocks[blocks.length - 1]; 
               blocks[blocks.length - 1] = new BlockInfoUnderConstruction(
                   lastBlk, replication);
@@ -1023,7 +1023,6 @@ public class FSImageFormat {
     if (!NameNodeLayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
       if (Arrays.equals(component, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES)) {
         Preconditions.checkArgument(
-            renameReservedMap != null &&
             renameReservedMap.containsKey(HdfsConstants.DOT_SNAPSHOT_DIR),
             RESERVED_ERROR_MSG);
         component =
@@ -1044,7 +1043,6 @@ public class FSImageFormat {
     if (!NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) {
       if (Arrays.equals(component, FSDirectory.DOT_RESERVED)) {
         Preconditions.checkArgument(
-            renameReservedMap != null &&
             renameReservedMap.containsKey(FSDirectory.DOT_RESERVED_STRING),
             RESERVED_ERROR_MSG);
         final String renameString = renameReservedMap

+ 4 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1095,9 +1095,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       stopSecretManager();
-      if (leaseManager != null) {
-        leaseManager.stopMonitor();
-      }
+      leaseManager.stopMonitor();
       if (nnrmthread != null) {
         ((NameNodeResourceMonitor) nnrmthread.getRunnable()).stopMonitor();
         nnrmthread.interrupt();
@@ -3775,20 +3773,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     final BlockInfo lastBlock = pendingFile.getLastBlock();
     BlockUCState lastBlockState = lastBlock.getBlockUCState();
     BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
-    boolean penultimateBlockMinReplication;
-    BlockUCState penultimateBlockState;
-    if (penultimateBlock == null) {
-      penultimateBlockState = BlockUCState.COMPLETE;
-      // If penultimate block doesn't exist then its minReplication is met
-      penultimateBlockMinReplication = true;
-    } else {
-      penultimateBlockState = BlockUCState.COMMITTED;
-      penultimateBlockMinReplication = 
+
+    // If penultimate block doesn't exist then its minReplication is met
+    boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
         blockManager.checkMinReplication(penultimateBlock);
-    }
-    assert penultimateBlockState == BlockUCState.COMPLETE ||
-           penultimateBlockState == BlockUCState.COMMITTED :
-           "Unexpected state of penultimate block in " + src;
 
     switch(lastBlockState) {
     case COMPLETE:

+ 13 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -435,17 +435,19 @@ public class INodeFile extends INodeWithAdditionalFields
           removedINodes, countDiffChange);
     }
     Quota.Counts counts = Quota.Counts.newInstance();
-    if (snapshot == CURRENT_STATE_ID && priorSnapshotId == NO_SNAPSHOT_ID) {
-      // this only happens when deleting the current file and the file is not
-      // in any snapshot
-      computeQuotaUsage(counts, false);
-      destroyAndCollectBlocks(collectedBlocks, removedINodes);
-    } else if (snapshot == CURRENT_STATE_ID && priorSnapshotId != NO_SNAPSHOT_ID) {
-      // when deleting the current file and the file is in snapshot, we should
-      // clean the 0-sized block if the file is UC
-      FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
-      if (uc != null) {
-        uc.cleanZeroSizeBlock(this, collectedBlocks);
+    if (snapshot == CURRENT_STATE_ID) {
+      if (priorSnapshotId == NO_SNAPSHOT_ID) {
+        // this only happens when deleting the current file and the file is not
+        // in any snapshot
+        computeQuotaUsage(counts, false);
+        destroyAndCollectBlocks(collectedBlocks, removedINodes);
+      } else {
+        // when deleting the current file and the file is in snapshot, we should
+        // clean the 0-sized block if the file is UC
+        FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
+        if (uc != null) {
+          uc.cleanZeroSizeBlock(this, collectedBlocks);
+        }
       }
     }
     return counts;

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

@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
-import org.apache.hadoop.classification.InterfaceAudience;;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 @InterfaceAudience.Private
 public class SaveNamespaceCancelledException extends IOException {