Explorar o código

HDFS-12884. BlockUnderConstructionFeature.truncateBlock should be of type BlockInfo. Contributed by chencan.

(cherry picked from commit 8d898ab25f1c2032a07c9bbd96ba3d0c4eb5be87)
Konstantin V Shvachko %!s(int64=7) %!d(string=hai) anos
pai
achega
37403e1755

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

@@ -57,7 +57,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /**
   /**
    * The block source to use in the event of copy-on-write truncate.
    * The block source to use in the event of copy-on-write truncate.
    */
    */
-  private Block truncateBlock;
+  private BlockInfoContiguous truncateBlock;
 
 
   /**
   /**
    * ReplicaUnderConstruction contains information about replicas while
    * ReplicaUnderConstruction contains information about replicas while
@@ -233,11 +233,11 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   }
   }
 
 
   /** Get recover block */
   /** Get recover block */
-  public Block getTruncateBlock() {
+  public BlockInfoContiguous getTruncateBlock() {
     return truncateBlock;
     return truncateBlock;
   }
   }
 
 
-  public void setTruncateBlock(Block recoveryBlock) {
+  public void setTruncateBlock(BlockInfoContiguous recoveryBlock) {
     this.truncateBlock = recoveryBlock;
     this.truncateBlock = recoveryBlock;
   }
   }
 
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2192,7 +2192,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       oldBlock = file.getLastBlock();
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
       truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
       truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
-      truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
+      truncatedBlockUC.setTruncateBlock(new BlockInfoContiguous(oldBlock,
+          file.getBlockReplication()));
       truncatedBlockUC.getTruncateBlock().setNumBytes(
       truncatedBlockUC.getTruncateBlock().setNumBytes(
           oldBlock.getNumBytes() - lastBlockDelta);
           oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.getTruncateBlock().setGenerationStamp(
       truncatedBlockUC.getTruncateBlock().setGenerationStamp(
@@ -4069,7 +4070,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     case UNDER_RECOVERY:
     case UNDER_RECOVERY:
       final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
       final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       // determine if last block was intended to be truncated
-      Block recoveryBlock = uc.getTruncateBlock();
+      BlockInfoContiguous recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
       boolean truncateRecovery = recoveryBlock != null;
       boolean copyOnTruncate = truncateRecovery &&
       boolean copyOnTruncate = truncateRecovery &&
           recoveryBlock.getBlockId() != uc.getBlockId();
           recoveryBlock.getBlockId() != uc.getBlockId();

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

@@ -879,14 +879,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
       if(!(toDelete instanceof BlockInfoContiguousUnderConstruction)) {
       if(!(toDelete instanceof BlockInfoContiguousUnderConstruction)) {
         return;
         return;
       }
       }
-      Block truncateBlock =
+      BlockInfoContiguous truncateBlock =
           ((BlockInfoContiguousUnderConstruction)toDelete).getTruncateBlock();
           ((BlockInfoContiguousUnderConstruction)toDelete).getTruncateBlock();
       if(truncateBlock == null || truncateBlock.equals(toDelete)) {
       if(truncateBlock == null || truncateBlock.equals(toDelete)) {
         return;
         return;
       }
       }
-      assert truncateBlock instanceof BlockInfoContiguous :
-        "should be BlockInfoContiguous";
-      addDeleteBlock((BlockInfoContiguous) truncateBlock);
+      addDeleteBlock(truncateBlock);
     }
     }
 
 
     public void removeDeleteBlock(Block block) {
     public void removeDeleteBlock(Block block) {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
 
 
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -130,7 +129,7 @@ public class FileDiffList extends
     }
     }
     // Check if last block is part of truncate recovery
     // Check if last block is part of truncate recovery
     BlockInfoContiguous lastBlock = file.getLastBlock();
     BlockInfoContiguous lastBlock = file.getLastBlock();
-    Block dontRemoveBlock = null;
+    BlockInfoContiguous dontRemoveBlock = null;
     if(lastBlock != null && lastBlock.getBlockUCState().equals(
     if(lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
       dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
       dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)