Browse Source

HDFS-265. Merge -c 820487 https://svn.apache.org/repos/asf/hadoop/hdfs/branches/branch-0.21

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@820497 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 years ago
parent
commit
58f1184e1a
95 changed files with 7003 additions and 1668 deletions
  1. 93 0
      CHANGES.txt
  2. 1 1
      src/docs/src/documentation/content/xdocs/hdfs_imageviewer.xml
  3. 424 261
      src/java/org/apache/hadoop/hdfs/DFSClient.java
  4. 174 43
      src/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
  5. 4 13
      src/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  6. 44 6
      src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  7. 73 13
      src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java
  8. 4 2
      src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
  9. 17 0
      src/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  10. 49 2
      src/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  11. 31 0
      src/java/org/apache/hadoop/hdfs/protocol/RecoveryInProgressException.java
  12. 76 0
      src/java/org/apache/hadoop/hdfs/server/common/HdfsConstants.java
  13. 3 0
      src/java/org/apache/hadoop/hdfs/server/common/Storage.java
  14. 128 116
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  15. 46 13
      src/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  16. 3 3
      src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  17. 192 130
      src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  18. 61 3
      src/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  19. 38 20
      src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  20. 555 332
      src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  21. 103 7
      src/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  22. 95 0
      src/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
  23. 55 0
      src/java/org/apache/hadoop/hdfs/server/datanode/Replica.java
  24. 3 3
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java
  25. 88 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
  26. 182 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
  27. 60 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java
  28. 140 45
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  29. 53 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
  30. 163 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
  31. 101 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
  32. 115 0
      src/java/org/apache/hadoop/hdfs/server/datanode/ReplicasMap.java
  33. 51 3
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java
  34. 266 0
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java
  35. 170 47
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  36. 22 4
      src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
  37. 165 40
      src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
  38. 27 13
      src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  39. 23 19
      src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  40. 11 3
      src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  41. 402 112
      src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  42. 0 8
      src/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  43. 43 6
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  44. 22 63
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  45. 26 4
      src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  46. 20 7
      src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  47. 161 0
      src/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java
  48. 2 9
      src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
  49. 23 3
      src/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
  50. 81 0
      src/java/org/apache/hadoop/hdfs/server/protocol/ReplicaRecoveryInfo.java
  51. 1 1
      src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  52. 65 0
      src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java
  53. 13 12
      src/test/aop/org/apache/hadoop/hdfs/DFSClientAspects.aj
  54. 57 0
      src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj
  55. 175 0
      src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java
  56. 1 1
      src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj
  57. 3 2
      src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
  58. 4 6
      src/test/aop/org/apache/hadoop/hdfs/server/datanode/FSDatasetAspects.aj
  59. 10 5
      src/test/findbugsExcludeFile.xml
  60. 12 5
      src/test/hdfs/org/apache/hadoop/hdfs/AppendTestUtil.java
  61. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
  62. 2 1
      src/test/hdfs/org/apache/hadoop/hdfs/DataNodeCluster.java
  63. 10 7
      src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
  64. 3 3
      src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java
  65. 12 11
      src/test/hdfs/org/apache/hadoop/hdfs/TestBlockReport.java
  66. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
  67. 118 0
      src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
  68. 8 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  69. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java
  70. 191 47
      src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
  71. 10 10
      src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
  72. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java
  73. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java
  74. 2 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java
  75. 4 5
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java
  76. 146 0
      src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java
  77. 4 5
      src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
  78. 23 36
      src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  79. 5 4
      src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
  80. 149 0
      src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java
  81. 9 4
      src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
  82. 3 2
      src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java
  83. 3 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  84. 204 29
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  85. 4 6
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
  86. 204 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java
  87. 19 28
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  88. 13 15
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
  89. 151 15
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
  90. 31 24
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
  91. 431 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java
  92. 25 14
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  93. 176 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  94. 4 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  95. 2 1
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java

+ 93 - 0
CHANGES.txt

@@ -26,6 +26,21 @@ Release 0.21.0 - Unreleased
     HDFS-602. DistributedFileSystem mkdirs throws FileAlreadyExistsException
     HDFS-602. DistributedFileSystem mkdirs throws FileAlreadyExistsException
     instead of FileNotFoundException. (Boris Shkolnik via suresh)
     instead of FileNotFoundException. (Boris Shkolnik via suresh)
 
 
+    HDFS-544. Add a "rbw" subdir to DataNode data directory. (hairong)
+
+    HDFS-576. Block report includes under-construction replicas. (shv)
+
+    HDFS-636. SafeMode counts complete blocks only. (shv)
+
+    HDFS-644. Lease recovery, concurrency support. (shv)
+
+    HDFS-570. Get last block length from a data-node when opening a file
+    being written to. (Tsz Wo (Nicholas), SZE via shv)
+
+    HDFS-657. Remove unused legacy data-node protocol methods. (shv)
+
+    HDFS-658. Block recovery for primary data-node. (shv)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HDFS-436. Introduce AspectJ framework for HDFS code and tests.
     HDFS-436. Introduce AspectJ framework for HDFS code and tests.
@@ -64,6 +79,37 @@ Release 0.21.0 - Unreleased
 
 
     HDFS-610. Support o.a.h.fs.FileContext.  (Sanjay Radia via szetszwo)
     HDFS-610. Support o.a.h.fs.FileContext.  (Sanjay Radia via szetszwo)
 
 
+    HDFS-536. Support hflush at DFSClient. (hairong)
+
+    HDFS-517. Introduce BlockInfoUnderConstruction to reflect block replica
+    states while writing. (shv)
+
+    HDFS-565. Introduce block committing logic during new block allocation
+    and file close. (shv)
+
+    HDFS-537. DataNode exposes a replica's meta info to BlockReceiver for the
+    support of dfs writes/hflush. It also updates a replica's bytes received,
+    bytes on disk, and bytes acked after receiving a packet. (hairong)
+
+    HDFS-585. Datanode should serve up to visible length of a replica for read
+    requests.  (szetszwo)
+
+    HDFS-604. Block report processing for append. (shv)
+
+    HDFS-619. Support replica recovery initialization in datanode for the new
+    append design.  (szetszwo)
+
+    HDFS-592. Allow clients to fetch a new generation stamp from NameNode for
+    pipeline recovery. (hairong)
+
+    HDFS-624. Support a new algorithm for pipeline recovery and pipeline setup
+    for append. (hairong)
+
+    HDFS-627. Support replica update in data-node.
+    (Tsz Wo (Nicholas), SZE and Hairong Kuang via shv)
+
+    HDFS-642. Support pipeline close and close error recovery. (hairong)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-381. Remove blocks from DataNode maps when corresponding file
     HDFS-381. Remove blocks from DataNode maps when corresponding file
@@ -191,6 +237,31 @@ Release 0.21.0 - Unreleased
     HDFS-641. Move all of the components that depend on map/reduce to 
     HDFS-641. Move all of the components that depend on map/reduce to 
     map/reduce. (omalley)
     map/reduce. (omalley)
 
 
+    HDFS-509. Redesign DataNode volumeMap to include all types of Replicas.
+    (hairong)
+
+    HDFS-562. Add a test for NameNode.getBlockLocations(..) to check read from
+    un-closed file.  (szetszwo)
+
+    HDFS-543. Break FSDatasetInterface#writToBlock() into writeToRemporary,
+    writeToRBW, ad append. (hairong)
+
+    HDFS-603. Add a new interface, Replica, which is going to replace the use
+    of Block in datanode.  (szetszwo)
+
+    HDFS-589. Change block write protocol to support pipeline recovery.
+    (hairong)
+
+    HDFS-652. Replace BlockInfo.isUnderConstruction() with isComplete() (shv)
+
+    HDFS-648. Change some methods in AppendTestUtil to public.  (Konstantin
+    Boudnik via szetszwo)
+
+    HDFS-662. Unnecessary info message from DFSClient. (hairong)
+
+    HDFS-518. Create new tests for Append's hflush. (Konstantin Boudnik
+    via szetszwo)
+
   BUG FIXES
   BUG FIXES
 
 
     HDFS-76. Better error message to users when commands fail because of 
     HDFS-76. Better error message to users when commands fail because of 
@@ -288,6 +359,28 @@ Release 0.21.0 - Unreleased
 
 
     HDFS-640. Fixed TestHDFSFileContextMainOperations.java build failure. (suresh)
     HDFS-640. Fixed TestHDFSFileContextMainOperations.java build failure. (suresh)
 
 
+    HDFS-547. TestHDFSFileSystemContract#testOutputStreamClosedTwice
+    sometimes fails with CloseByInterruptException. (hairong)
+
+    HDFS-588. Fix TestFiDataTransferProtocol and TestAppend2 failures. (shv)
+
+    HDFS-550. DataNode restarts may introduce corrupt/duplicated/lost replicas
+    when handling detached replicas. (hairong)
+
+    HDFS-659. If the the last block is not complete, update its length with
+    one of its replica's length stored in datanode.  (szetszwo)
+
+    HDFS-649. Check null pointers for DataTransferTest.  (Konstantin Boudnik
+    via szetszwo)
+
+    HDFS-661. DataNode upgrade fails on non-existant current directory.
+    (hairong)
+
+    HDFS-597. Mofication introduced by HDFS-537 breakes an advice binding in
+    FSDatasetAspects.  (Konstantin Boudnik via szetszwo)
+
+    HDFS-665. TestFileAppend2 sometimes hangs. (hairong)
+
 Release 0.20.1 - 2009-09-01
 Release 0.20.1 - 2009-09-01
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 1 - 1
src/docs/src/documentation/content/xdocs/hdfs_imageviewer.xml

@@ -219,7 +219,7 @@ FSImage
 
 
         PermString = rwxr-xr-x 
         PermString = rwxr-xr-x 
 
 
-…remaining output omitted…
+���remaining output omitted���
 </source>          
 </source>          
           
           
       </section> <!-- example-->
       </section> <!-- example-->

File diff suppressed because it is too large
+ 424 - 261
src/java/org/apache/hadoop/hdfs/DFSClient.java


+ 174 - 43
src/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java

@@ -18,6 +18,10 @@
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
 import java.util.Iterator;
 import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 
 
 /**
 /**
  * This class provides an interface for accessing list of blocks that
  * This class provides an interface for accessing list of blocks that
@@ -25,41 +29,82 @@ import java.util.Iterator;
  * This class is useful for block report. Rather than send block reports
  * This class is useful for block report. Rather than send block reports
  * as a Block[] we can send it as a long[].
  * as a Block[] we can send it as a long[].
  *
  *
+ * The structure of the array is as follows:
+ * 0: the length of the finalized replica list;
+ * 1: the length of the under-construction replica list;
+ * - followed by finalized replica list where each replica is represented by
+ *   3 longs: one for the blockId, one for the block length, and one for
+ *   the generation stamp;
+ * - followed by the invalid replica represented with three -1s;
+ * - followed by the under-construction replica list where each replica is
+ *   represented by 4 longs: three for the block id, length, generation 
+ *   stamp, and the forth for the replica state.
  */
  */
-public class BlockListAsLongs implements Iterable<Block>{
+public class BlockListAsLongs implements Iterable<Block> {
   /**
   /**
-   * A block as 3 longs
+   * A finalized block as 3 longs
    *   block-id and block length and generation stamp
    *   block-id and block length and generation stamp
    */
    */
-  private static final int LONGS_PER_BLOCK = 3;
-  
-  private static int index2BlockId(int index) {
-    return index*LONGS_PER_BLOCK;
-  }
-  private static int index2BlockLen(int index) {
-    return (index*LONGS_PER_BLOCK) + 1;
-  }
-  private static int index2BlockGenStamp(int index) {
-    return (index*LONGS_PER_BLOCK) + 2;
+  private static final int LONGS_PER_FINALIZED_BLOCK = 3;
+
+  /**
+   * An under-construction block as 4 longs
+   *   block-id and block length, generation stamp and replica state
+   */
+  private static final int LONGS_PER_UC_BLOCK = 4;
+
+  /** Number of longs in the header */
+  private static final int HEADER_SIZE = 2;
+
+  /**
+   * Returns the index of the first long in blockList
+   * belonging to the specified block.
+   * The first long contains the block id.
+   */
+  private int index2BlockId(int blockIndex) {
+    if(blockIndex < 0 || blockIndex > getNumberOfBlocks())
+      return -1;
+    int finalizedSize = getNumberOfFinalizedReplicas();
+    if(blockIndex < finalizedSize)
+      return HEADER_SIZE + blockIndex * LONGS_PER_FINALIZED_BLOCK;
+    return HEADER_SIZE + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK
+            + (blockIndex - finalizedSize) * LONGS_PER_UC_BLOCK;
   }
   }
-  
+
   private long[] blockList;
   private long[] blockList;
   
   
   /**
   /**
-   * Converting a block[] to a long[]
-   * @param blockArray - the input array block[]
-   * @return the output array of long[]
+   * Create block report from finalized and under construction lists of blocks.
+   * 
+   * @param finalized - list of finalized blocks
+   * @param uc - list of under construction blocks
    */
    */
-  public static long[] convertToArrayLongs(final Block[] blockArray) {
-    long[] blocksAsLongs = new long[blockArray.length * LONGS_PER_BLOCK];
+  public BlockListAsLongs(final List<? extends Block> finalized,
+                          final List<ReplicaInfo> uc) {
+    int finalizedSize = finalized == null ? 0 : finalized.size();
+    int ucSize = uc == null ? 0 : uc.size();
+    int len = HEADER_SIZE
+              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK
+              + ucSize * LONGS_PER_UC_BLOCK;
+
+    blockList = new long[len];
 
 
-    BlockListAsLongs bl = new BlockListAsLongs(blocksAsLongs);
-    assert bl.getNumberOfBlocks() == blockArray.length;
+    // set the header
+    blockList[0] = finalizedSize;
+    blockList[1] = ucSize;
 
 
-    for (int i = 0; i < blockArray.length; i++) {
-      bl.setBlock(i, blockArray[i]);
+    // set finalized blocks
+    for (int i = 0; i < finalizedSize; i++) {
+      setBlock(i, finalized.get(i));
+    }
+
+    // set invalid delimiting block
+    setDelimitingBlock(finalizedSize);
+
+    // set under construction blocks
+    for (int i = 0; i < ucSize; i++) {
+      setBlock(finalizedSize + i, uc.get(i));
     }
     }
-    return blocksAsLongs;
   }
   }
 
 
   public BlockListAsLongs() {
   public BlockListAsLongs() {
@@ -72,27 +117,29 @@ public class BlockListAsLongs implements Iterable<Block>{
    */
    */
   public BlockListAsLongs(final long[] iBlockList) {
   public BlockListAsLongs(final long[] iBlockList) {
     if (iBlockList == null) {
     if (iBlockList == null) {
-      blockList = new long[0];
-    } else {
-      if (iBlockList.length%LONGS_PER_BLOCK != 0) {
-        // must be multiple of LONGS_PER_BLOCK
-        throw new IllegalArgumentException();
-      }
-      blockList = iBlockList;
+      blockList = new long[HEADER_SIZE];
+      return;
     }
     }
+    blockList = iBlockList;
+  }
+
+  public long[] getBlockListAsLongs() {
+    return blockList;
   }
   }
 
 
   /**
   /**
    * Iterates over blocks in the block report.
    * Iterates over blocks in the block report.
    * Avoids object allocation on each iteration.
    * Avoids object allocation on each iteration.
    */
    */
-  private class BlockReportIterator implements Iterator<Block> {
+  public class BlockReportIterator implements Iterator<Block> {
     private int currentBlockIndex;
     private int currentBlockIndex;
     private Block block;
     private Block block;
+    private ReplicaState currentReplicaState;
 
 
     BlockReportIterator() {
     BlockReportIterator() {
       this.currentBlockIndex = 0;
       this.currentBlockIndex = 0;
       this.block = new Block();
       this.block = new Block();
+      this.currentReplicaState = null;
     }
     }
 
 
     public boolean hasNext() {
     public boolean hasNext() {
@@ -100,22 +147,39 @@ public class BlockListAsLongs implements Iterable<Block>{
     }
     }
 
 
     public Block next() {
     public Block next() {
-      block.set(blockList[index2BlockId(currentBlockIndex)],
-                blockList[index2BlockLen(currentBlockIndex)],
-                blockList[index2BlockGenStamp(currentBlockIndex)]);
+      block.set(blockId(currentBlockIndex),
+                blockLength(currentBlockIndex),
+                blockGenerationStamp(currentBlockIndex));
+      currentReplicaState = blockReplicaState(currentBlockIndex);
       currentBlockIndex++;
       currentBlockIndex++;
       return block;
       return block;
     }
     }
 
 
-    public void remove()  {
+    public void remove() {
       throw new UnsupportedOperationException("Sorry. can't remove.");
       throw new UnsupportedOperationException("Sorry. can't remove.");
     }
     }
+
+    /**
+     * Get the state of the current replica.
+     * The state corresponds to the replica returned
+     * by the latest {@link #next()}. 
+     */
+    public ReplicaState getCurrentReplicaState() {
+      return currentReplicaState;
+    }
   }
   }
 
 
   /**
   /**
    * Returns an iterator over blocks in the block report. 
    * Returns an iterator over blocks in the block report. 
    */
    */
   public Iterator<Block> iterator() {
   public Iterator<Block> iterator() {
+    return getBlockReportIterator();
+  }
+
+  /**
+   * Returns {@link BlockReportIterator}. 
+   */
+  public BlockReportIterator getBlockReportIterator() {
     return new BlockReportIterator();
     return new BlockReportIterator();
   }
   }
 
 
@@ -124,7 +188,55 @@ public class BlockListAsLongs implements Iterable<Block>{
    * @return - the number of blocks
    * @return - the number of blocks
    */
    */
   public int getNumberOfBlocks() {
   public int getNumberOfBlocks() {
-    return blockList.length/LONGS_PER_BLOCK;
+    assert blockList.length == HEADER_SIZE + 
+            (blockList[0] + 1) * LONGS_PER_FINALIZED_BLOCK +
+            blockList[1] * LONGS_PER_UC_BLOCK :
+              "Number of blocks is inconcistent with the array length";
+    return getNumberOfFinalizedReplicas() + getNumberOfUCReplicas();
+  }
+
+  /**
+   * Returns the number of finalized replicas in the block report.
+   */
+  private int getNumberOfFinalizedReplicas() {
+    return (int)blockList[0];
+  }
+
+  /**
+   * Returns the number of under construction replicas in the block report.
+   */
+  private int getNumberOfUCReplicas() {
+    return (int)blockList[1];
+  }
+
+  /**
+   * Returns the id of the specified replica of the block report.
+   */
+  private long blockId(int index) {
+    return blockList[index2BlockId(index)];
+  }
+
+  /**
+   * Returns the length of the specified replica of the block report.
+   */
+  private long blockLength(int index) {
+    return blockList[index2BlockId(index) + 1];
+  }
+
+  /**
+   * Returns the generation stamp of the specified replica of the block report.
+   */
+  private long blockGenerationStamp(int index) {
+    return blockList[index2BlockId(index) + 2];
+  }
+
+  /**
+   * Returns the state of the specified replica of the block report.
+   */
+  private ReplicaState blockReplicaState(int index) {
+    if(index < getNumberOfFinalizedReplicas())
+      return ReplicaState.FINALIZED;
+    return ReplicaState.getState((int)blockList[index2BlockId(index) + 3]);
   }
   }
 
 
   /**
   /**
@@ -134,7 +246,7 @@ public class BlockListAsLongs implements Iterable<Block>{
    */
    */
   @Deprecated
   @Deprecated
   public long getBlockId(final int index)  {
   public long getBlockId(final int index)  {
-    return blockList[index2BlockId(index)];
+    return blockId(index);
   }
   }
   
   
   /**
   /**
@@ -144,7 +256,7 @@ public class BlockListAsLongs implements Iterable<Block>{
    */
    */
   @Deprecated
   @Deprecated
   public long getBlockLen(final int index)  {
   public long getBlockLen(final int index)  {
-    return blockList[index2BlockLen(index)];
+    return blockLength(index);
   }
   }
 
 
   /**
   /**
@@ -154,7 +266,7 @@ public class BlockListAsLongs implements Iterable<Block>{
    */
    */
   @Deprecated
   @Deprecated
   public long getBlockGenStamp(final int index)  {
   public long getBlockGenStamp(final int index)  {
-    return blockList[index2BlockGenStamp(index)];
+    return blockGenerationStamp(index);
   }
   }
   
   
   /**
   /**
@@ -162,9 +274,28 @@ public class BlockListAsLongs implements Iterable<Block>{
    * @param index - the index of the block to set
    * @param index - the index of the block to set
    * @param b - the block is set to the value of the this block
    * @param b - the block is set to the value of the this block
    */
    */
-  private void setBlock(final int index, final Block b) {
-    blockList[index2BlockId(index)] = b.getBlockId();
-    blockList[index2BlockLen(index)] = b.getNumBytes();
-    blockList[index2BlockGenStamp(index)] = b.getGenerationStamp();
+  private <T extends Block> void setBlock(final int index, final T b) {
+    int pos = index2BlockId(index);
+    blockList[pos] = b.getBlockId();
+    blockList[pos + 1] = b.getNumBytes();
+    blockList[pos + 2] = b.getGenerationStamp();
+    if(index < getNumberOfFinalizedReplicas())
+      return;
+    assert ((ReplicaInfo)b).getState() != ReplicaState.FINALIZED :
+      "Must be under-construction replica.";
+    blockList[pos + 3] = ((ReplicaInfo)b).getState().getValue();
+  }
+
+  /**
+   * Set the invalid delimiting block between the finalized and
+   * the under-construction lists.
+   * The invalid block has all three fields set to -1.
+   * @param finalizedSzie - the size of the finalized list
+   */
+  private void setDelimitingBlock(final int finalizedSzie) {
+    int idx = HEADER_SIZE + finalizedSzie * LONGS_PER_FINALIZED_BLOCK;
+    blockList[idx] = -1;
+    blockList[idx+1] = -1;
+    blockList[idx+2] = -1;
   }
   }
 }
 }

+ 4 - 13
src/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -29,19 +29,10 @@ public interface ClientDatanodeProtocol extends VersionedProtocol {
   public static final Log LOG = LogFactory.getLog(ClientDatanodeProtocol.class);
   public static final Log LOG = LogFactory.getLog(ClientDatanodeProtocol.class);
 
 
   /**
   /**
-   * 4: never return null and always return a newly generated access token
+   * 6: recoverBlock() removed.
    */
    */
-  public static final long versionID = 4L;
+  public static final long versionID = 6L;
 
 
-  /** Start generation-stamp recovery for specified block
-   * @param block the specified block
-   * @param keepLength keep the block length
-   * @param targets the list of possible locations of specified block
-   * @return either a new generation stamp, or the original generation stamp. 
-   * Regardless of whether a new generation stamp is returned, a newly 
-   * generated access token is returned as part of the return value.
-   * @throws IOException
-   */
-  LocatedBlock recoverBlock(Block block, boolean keepLength,
-      DatanodeInfo[] targets) throws IOException;
+  /** Return the visible length of a replica. */
+  long getReplicaVisibleLength(Block b) throws IOException;
 }
 }

+ 44 - 6
src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -44,9 +44,9 @@ public interface ClientProtocol extends VersionedProtocol {
    * Compared to the previous version the following changes have been introduced:
    * Compared to the previous version the following changes have been introduced:
    * (Only the latest change is reflected.
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
    * The log of historical changes can be retrieved from the svn).
-   * 48: modified mkdirs() to take an additional boolean parameter
+   * 50: change LocatedBlocks to include last block information.
    */
    */
-  public static final long versionID = 48L;
+  public static final long versionID = 50L;
   
   
   ///////////////////////////////////////
   ///////////////////////////////////////
   // File contents
   // File contents
@@ -93,8 +93,8 @@ public interface ClientProtocol extends VersionedProtocol {
    * {@link #rename(String, String)} it until the file is completed
    * {@link #rename(String, String)} it until the file is completed
    * or explicitly as a result of lease expiration.
    * or explicitly as a result of lease expiration.
    * <p>
    * <p>
-   * Blocks have a maximum size.  Clients that intend to
-   * create multi-block files must also use {@link #addBlock(String, String)}.
+   * Blocks have a maximum size.  Clients that intend to create
+   * multi-block files must also use {@link #addBlock(String, String, Block)}.
    *
    *
    * @param src path of the file being created.
    * @param src path of the file being created.
    * @param masked masked permission.
    * @param masked masked permission.
@@ -187,9 +187,14 @@ public interface ClientProtocol extends VersionedProtocol {
    * addBlock() allocates a new block and datanodes the block data
    * addBlock() allocates a new block and datanodes the block data
    * should be replicated to.
    * should be replicated to.
    * 
    * 
+   * addBlock() also commits the previous block by reporting
+   * to the name-node the actual generation stamp and the length
+   * of the block that the client has transmitted to data-nodes.
+   * 
    * @return LocatedBlock allocated block information.
    * @return LocatedBlock allocated block information.
    */
    */
-  public LocatedBlock addBlock(String src, String clientName) throws IOException;
+  public LocatedBlock addBlock(String src, String clientName,
+                               Block previous) throws IOException;
 
 
   /**
   /**
    * The client is done writing data to the given filename, and would 
    * The client is done writing data to the given filename, and would 
@@ -197,13 +202,18 @@ public interface ClientProtocol extends VersionedProtocol {
    *
    *
    * The function returns whether the file has been closed successfully.
    * The function returns whether the file has been closed successfully.
    * If the function returns false, the caller should try again.
    * If the function returns false, the caller should try again.
+   * 
+   * close() also commits the last block of the file by reporting
+   * to the name-node the actual generation stamp and the length
+   * of the block that the client has transmitted to data-nodes.
    *
    *
    * A call to complete() will not return true until all the file's
    * A call to complete() will not return true until all the file's
    * blocks have been replicated the minimum number of times.  Thus,
    * blocks have been replicated the minimum number of times.  Thus,
    * DataNode failures may cause a client to call complete() several
    * DataNode failures may cause a client to call complete() several
    * times before succeeding.
    * times before succeeding.
    */
    */
-  public boolean complete(String src, String clientName) throws IOException;
+  public boolean complete(String src, String clientName,
+                          Block last) throws IOException;
 
 
   /**
   /**
    * The client wants to report corrupted blocks (blocks with specified
    * The client wants to report corrupted blocks (blocks with specified
@@ -500,4 +510,32 @@ public interface ClientProtocol extends VersionedProtocol {
    *              by this call.
    *              by this call.
    */
    */
   public void setTimes(String src, long mtime, long atime) throws IOException;
   public void setTimes(String src, long mtime, long atime) throws IOException;
+  
+  /**
+   * Get a new generation stamp together with an access token for 
+   * a block under construction
+   * 
+   * This method is called only when a client needs to recover a failed
+   * pipeline or set up a pipeline for appending to a block.
+   * 
+   * @param block a block
+   * @param clientName the name of the client
+   * @return a located block with a new generation stamp and an access token
+   * @throws IOException if any error occurs
+   */
+  public LocatedBlock updateBlockForPipeline(Block block, String clientName) 
+  throws IOException;
+
+  /**
+   * Update a pipeline for a block under construction
+   * 
+   * @param clientName the name of the client
+   * @param oldBlock the old block
+   * @param newBlock the new block containing new generation stamp and length
+   * @param newNodes datanodes in the pipeline
+   * @throws IOException if any error occurs
+   */
+  public void updatePipeline(String clientName, Block oldBlock, 
+      Block newBlock, DatanodeID[] newNodes)
+  throws IOException;
 }
 }

+ 73 - 13
src/java/org/apache/hadoop/hdfs/protocol/DataTransferProtocol.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
 
 
 /**
 /**
@@ -38,12 +39,12 @@ public interface DataTransferProtocol {
    * when protocol changes. It is not very obvious. 
    * when protocol changes. It is not very obvious. 
    */
    */
   /*
   /*
-   * Version 16:
-   *    Datanode now needs to send back a status code together 
-   *    with firstBadLink during pipeline setup for dfs write
-   *    (only for DFSClients, not for other datanodes).
+   * Version 17:
+   *    Change the block write protocol to support pipeline recovery.
+   *    Additional fields, like recovery flags, new GS, minBytesRcvd, 
+   *    and maxBytesRcvd are included.
    */
    */
-  public static final int DATA_TRANSFER_VERSION = 16;
+  public static final int DATA_TRANSFER_VERSION = 17;
 
 
   /** Operation */
   /** Operation */
   public enum Op {
   public enum Op {
@@ -119,6 +120,55 @@ public interface DataTransferProtocol {
     }
     }
   };
   };
   
   
+  public enum BlockConstructionStage {
+    /** The enumerates are always listed as regular stage followed by the
+     * recovery stage. 
+     * Changing this order will make getRecoveryStage not working.
+     */
+    // pipeline set up for block append
+    PIPELINE_SETUP_APPEND,
+    // pipeline set up for failed PIPELINE_SETUP_APPEND recovery
+    PIPELINE_SETUP_APPEND_RECOVERY,
+    // data streaming
+    DATA_STREAMING,
+    // pipeline setup for failed data streaming recovery
+    PIPELINE_SETUP_STREAMING_RECOVERY,
+    // close the block and pipeline
+    PIPELINE_CLOSE,
+    // Recover a failed PIPELINE_CLOSE
+    PIPELINE_CLOSE_RECOVERY,
+    // pipeline set up for block creation
+    PIPELINE_SETUP_CREATE;
+    
+    final static private byte RECOVERY_BIT = (byte)1;
+    
+    /**
+     * get the recovery stage of this stage
+     */
+    public BlockConstructionStage getRecoveryStage() {
+      if (this == PIPELINE_SETUP_CREATE) {
+        throw new IllegalArgumentException( "Unexpected blockStage " + this);
+      } else {
+        return values()[ordinal()|RECOVERY_BIT];
+      }
+    }
+    
+    private static BlockConstructionStage valueOf(byte code) {
+      return code < 0 || code >= values().length? null: values()[code];
+    }
+    
+    /** Read from in */
+    private static BlockConstructionStage readFields(DataInput in)
+    throws IOException {
+      return valueOf(in.readByte());
+    }
+
+    /** write to out */
+    private void write(DataOutput out) throws IOException {
+      out.writeByte(ordinal());
+    }
+  }    
+
   /** @deprecated Deprecated at 0.21.  Use Op.WRITE_BLOCK instead. */
   /** @deprecated Deprecated at 0.21.  Use Op.WRITE_BLOCK instead. */
   @Deprecated
   @Deprecated
   public static final byte OP_WRITE_BLOCK = Op.WRITE_BLOCK.code;
   public static final byte OP_WRITE_BLOCK = Op.WRITE_BLOCK.code;
@@ -187,15 +237,19 @@ public interface DataTransferProtocol {
     
     
     /** Send OP_WRITE_BLOCK */
     /** Send OP_WRITE_BLOCK */
     public static void opWriteBlock(DataOutputStream out,
     public static void opWriteBlock(DataOutputStream out,
-        long blockId, long blockGs, int pipelineSize, boolean isRecovery,
-        String client, DatanodeInfo src, DatanodeInfo[] targets,
-        AccessToken accesstoken) throws IOException {
+        long blockId, long blockGs, int pipelineSize, 
+        BlockConstructionStage stage, long newGs, long minBytesRcvd,
+        long maxBytesRcvd, String client, DatanodeInfo src, 
+        DatanodeInfo[] targets, AccessToken accesstoken) throws IOException {
       op(out, Op.WRITE_BLOCK);
       op(out, Op.WRITE_BLOCK);
 
 
       out.writeLong(blockId);
       out.writeLong(blockId);
       out.writeLong(blockGs);
       out.writeLong(blockGs);
       out.writeInt(pipelineSize);
       out.writeInt(pipelineSize);
-      out.writeBoolean(isRecovery);
+      stage.write(out);
+      WritableUtils.writeVLong(out, newGs);
+      WritableUtils.writeVLong(out, minBytesRcvd);
+      WritableUtils.writeVLong(out, maxBytesRcvd);
       Text.writeString(out, client);
       Text.writeString(out, client);
 
 
       out.writeBoolean(src != null);
       out.writeBoolean(src != null);
@@ -307,7 +361,11 @@ public interface DataTransferProtocol {
       final long blockId = in.readLong();          
       final long blockId = in.readLong();          
       final long blockGs = in.readLong();
       final long blockGs = in.readLong();
       final int pipelineSize = in.readInt(); // num of datanodes in entire pipeline
       final int pipelineSize = in.readInt(); // num of datanodes in entire pipeline
-      final boolean isRecovery = in.readBoolean(); // is this part of recovery?
+      final BlockConstructionStage stage = 
+        BlockConstructionStage.readFields(in);
+      final long newGs = WritableUtils.readVLong(in);
+      final long minBytesRcvd = WritableUtils.readVLong(in);
+      final long maxBytesRcvd = WritableUtils.readVLong(in);
       final String client = Text.readString(in); // working on behalf of this client
       final String client = Text.readString(in); // working on behalf of this client
       final DatanodeInfo src = in.readBoolean()? DatanodeInfo.read(in): null;
       final DatanodeInfo src = in.readBoolean()? DatanodeInfo.read(in): null;
 
 
@@ -321,8 +379,8 @@ public interface DataTransferProtocol {
       }
       }
       final AccessToken accesstoken = readAccessToken(in);
       final AccessToken accesstoken = readAccessToken(in);
 
 
-      opWriteBlock(in, blockId, blockGs, pipelineSize, isRecovery,
-          client, src, targets, accesstoken);
+      opWriteBlock(in, blockId, blockGs, pipelineSize, stage,
+          newGs, minBytesRcvd, maxBytesRcvd, client, src, targets, accesstoken);
     }
     }
 
 
     /**
     /**
@@ -330,7 +388,9 @@ public interface DataTransferProtocol {
      * Write a block.
      * Write a block.
      */
      */
     protected abstract void opWriteBlock(DataInputStream in,
     protected abstract void opWriteBlock(DataInputStream in,
-        long blockId, long blockGs, int pipelineSize, boolean isRecovery,
+        long blockId, long blockGs,
+        int pipelineSize, BlockConstructionStage stage,
+        long newGs, long minBytesRcvd, long maxBytesRcvd,
         String client, DatanodeInfo src, DatanodeInfo[] targets,
         String client, DatanodeInfo src, DatanodeInfo[] targets,
         AccessToken accesstoken) throws IOException;
         AccessToken accesstoken) throws IOException;
 
 

+ 4 - 2
src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java

@@ -90,7 +90,9 @@ public interface FSConstants {
   // Version is reflected in the data storage file.
   // Version is reflected in the data storage file.
   // Versions are negative.
   // Versions are negative.
   // Decrement LAYOUT_VERSION to define a new version.
   // Decrement LAYOUT_VERSION to define a new version.
-  public static final int LAYOUT_VERSION = -19;
+  public static final int LAYOUT_VERSION = -20;
   // Current version: 
   // Current version: 
-  // -19: Sticky bit
+  // -20: DataNode adds a "rbw" sub directory to data directory
+  //      current dir contains "finalized" subdir for finalized replicas
+  //      and "rbw" subdir for replicas being written to.
 }
 }

+ 17 - 0
src/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -145,4 +145,21 @@ public class LocatedBlock implements Writable {
       locs[i].readFields(in);
       locs[i].readFields(in);
     }
     }
   }
   }
+
+  /** Read LocatedBlock from in. */
+  public static LocatedBlock read(DataInput in) throws IOException {
+    final LocatedBlock lb = new LocatedBlock();
+    lb.readFields(in);
+    return lb;
+  }
+
+  /** {@inheritDoc} */
+  public String toString() {
+    return getClass().getSimpleName() + "{" + b
+        + "; getBlockSize()=" + getBlockSize()
+        + "; corrupt=" + corrupt
+        + "; offset=" + offset
+        + "; locs=" + java.util.Arrays.asList(locs)
+        + "}";
+  }
 }
 }

+ 49 - 2
src/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -36,6 +36,8 @@ public class LocatedBlocks implements Writable {
   private long fileLength;
   private long fileLength;
   private List<LocatedBlock> blocks; // array of blocks with prioritized locations
   private List<LocatedBlock> blocks; // array of blocks with prioritized locations
   private boolean underConstruction;
   private boolean underConstruction;
+  private LocatedBlock lastLocatedBlock = null;
+  private boolean isLastBlockComplete = false;
 
 
   LocatedBlocks() {
   LocatedBlocks() {
     fileLength = 0;
     fileLength = 0;
@@ -43,11 +45,15 @@ public class LocatedBlocks implements Writable {
     underConstruction = false;
     underConstruction = false;
   }
   }
   
   
-  public LocatedBlocks(long flength, List<LocatedBlock> blks, boolean isUnderConstuction) {
-
+  /** public Constructor */
+  public LocatedBlocks(long flength, boolean isUnderConstuction,
+      List<LocatedBlock> blks, 
+      LocatedBlock lastBlock, boolean isLastBlockCompleted) {
     fileLength = flength;
     fileLength = flength;
     blocks = blks;
     blocks = blks;
     underConstruction = isUnderConstuction;
     underConstruction = isUnderConstuction;
+    this.lastLocatedBlock = lastBlock;
+    this.isLastBlockComplete = isLastBlockCompleted;
   }
   }
   
   
   /**
   /**
@@ -57,6 +63,16 @@ public class LocatedBlocks implements Writable {
     return blocks;
     return blocks;
   }
   }
   
   
+  /** Get the last located block. */
+  public LocatedBlock getLastLocatedBlock() {
+    return lastLocatedBlock;
+  }
+  
+  /** Is the last block completed? */
+  public boolean isLastBlockComplete() {
+    return isLastBlockComplete;
+  }
+
   /**
   /**
    * Get located block.
    * Get located block.
    */
    */
@@ -161,6 +177,15 @@ public class LocatedBlocks implements Writable {
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeLong(this.fileLength);
     out.writeLong(this.fileLength);
     out.writeBoolean(underConstruction);
     out.writeBoolean(underConstruction);
+
+    //write the last located block
+    final boolean isNull = lastLocatedBlock == null;
+    out.writeBoolean(isNull);
+    if (!isNull) {
+      lastLocatedBlock.write(out);
+    }
+    out.writeBoolean(isLastBlockComplete);
+
     // write located blocks
     // write located blocks
     int nrBlocks = locatedBlockCount();
     int nrBlocks = locatedBlockCount();
     out.writeInt(nrBlocks);
     out.writeInt(nrBlocks);
@@ -175,6 +200,14 @@ public class LocatedBlocks implements Writable {
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.fileLength = in.readLong();
     this.fileLength = in.readLong();
     underConstruction = in.readBoolean();
     underConstruction = in.readBoolean();
+
+    //read the last located block
+    final boolean isNull = in.readBoolean();
+    if (!isNull) {
+      lastLocatedBlock = LocatedBlock.read(in);
+    }
+    isLastBlockComplete = in.readBoolean();
+
     // read located blocks
     // read located blocks
     int nrBlocks = in.readInt();
     int nrBlocks = in.readInt();
     this.blocks = new ArrayList<LocatedBlock>(nrBlocks);
     this.blocks = new ArrayList<LocatedBlock>(nrBlocks);
@@ -184,4 +217,18 @@ public class LocatedBlocks implements Writable {
       this.blocks.add(blk);
       this.blocks.add(blk);
     }
     }
   }
   }
+
+  /** {@inheritDoc} */
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
+    b.append("{")
+     .append("\n  fileLength=").append(fileLength)
+     .append("\n  underConstruction=").append(underConstruction)
+     .append("\n  blocks=").append(blocks)
+     .append("\n  lastLocatedBlock=").append(lastLocatedBlock)
+     .append("\n  isLastBlockComplete=").append(isLastBlockComplete)
+     .append("}");
+    return b.toString();
+  }
 }
 }

+ 31 - 0
src/java/org/apache/hadoop/hdfs/protocol/RecoveryInProgressException.java

@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+/**
+ * Exception indicating that a replica is already being recovery.
+ */
+public class RecoveryInProgressException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public RecoveryInProgressException(String msg) {
+    super(msg);
+  }
+}

+ 76 - 0
src/java/org/apache/hadoop/hdfs/server/common/HdfsConstants.java

@@ -17,6 +17,10 @@
  */
  */
 package org.apache.hadoop.hdfs.server.common;
 package org.apache.hadoop.hdfs.server.common;
 
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 /************************************
 /************************************
  * Some handy internal HDFS constants
  * Some handy internal HDFS constants
  *
  *
@@ -80,5 +84,77 @@ public interface HdfsConstants {
       return description;
       return description;
     }
     }
   }
   }
+
+  /**
+   * Block replica states, which it can go through while being constructed.
+   */
+  static public enum ReplicaState {
+    /** Replica is finalized. The state when replica is not modified. */
+    FINALIZED(0),
+    /** Replica is being written to. */
+    RBW(1),
+    /** Replica is waiting to be recovered. */
+    RWR(2),
+    /** Replica is under recovery. */
+    RUR(3),
+    /** Temporary replica: created for replication and relocation only. */
+    TEMPORARY(4);
+
+    private int value;
+
+    private ReplicaState(int v) {
+      value = v;
+    }
+
+    public int getValue() {
+      return value;
+    }
+
+    public static ReplicaState getState(int v) {
+      return ReplicaState.values()[v];
+    }
+
+    /** Read from in */
+    public static ReplicaState read(DataInput in) throws IOException {
+      return values()[in.readByte()];
+    }
+
+    /** Write to out */
+    public void write(DataOutput out) throws IOException {
+      out.writeByte(ordinal());
+    }
+  }
+
+  /**
+   * States, which a block can go through while it is under construction.
+   */
+  static public enum BlockUCState {
+    /**
+     * Block construction completed.<br>
+     * The block has at least one {@link ReplicaState#FINALIZED} replica,
+     * and is not going to be modified.
+     */
+    COMPLETE,
+    /**
+     * The block is under construction.<br>
+     * It has been recently allocated for write or append.
+     */
+    UNDER_CONSTRUCTION,
+    /**
+     * The block is under recovery.<br>
+     * When a file lease expires its last block may not be {@link #COMPLETE}
+     * and needs to go through a recovery procedure, 
+     * which synchronizes the existing replicas contents.
+     */
+    UNDER_RECOVERY,
+    /**
+     * The block is committed.<br>
+     * The client reported that all bytes are written to data-nodes
+     * with the given generation stamp and block length, but no 
+     * {@link ReplicaState#FINALIZED} 
+     * replicas has yet been reported by data-nodes themselves.
+     */
+    COMMITTED;
+  }
 }
 }
 
 

+ 3 - 0
src/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -74,6 +74,9 @@ public abstract class Storage extends StorageInfo {
    * any upgrade code that uses this constant should also be removed. */
    * any upgrade code that uses this constant should also be removed. */
   public static final int PRE_GENERATIONSTAMP_LAYOUT_VERSION = -13;
   public static final int PRE_GENERATIONSTAMP_LAYOUT_VERSION = -13;
   
   
+  // last layout version that did not support persistent rbw replicas
+  public static final int PRE_RBW_LAYOUT_VERSION = -19;
+  
   private   static final String STORAGE_FILE_LOCK     = "in_use.lock";
   private   static final String STORAGE_FILE_LOCK     = "in_use.lock";
   protected static final String STORAGE_FILE_VERSION  = "VERSION";
   protected static final String STORAGE_FILE_VERSION  = "VERSION";
   public static final String STORAGE_DIR_CURRENT   = "current";
   public static final String STORAGE_DIR_CURRENT   = "current";

+ 128 - 116
src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Status;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
@@ -55,7 +56,6 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   
   
   private Block block; // the block to receive
   private Block block; // the block to receive
-  protected boolean finalized;
   private DataInputStream in = null; // from where data are read
   private DataInputStream in = null; // from where data are read
   private DataChecksum checksum; // from where chunks of a block can be read
   private DataChecksum checksum; // from where chunks of a block can be read
   private OutputStream out = null; // to block file at local disk
   private OutputStream out = null; // to block file at local disk
@@ -65,7 +65,6 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
   private ByteBuffer buf; // contains one full packet.
   private ByteBuffer buf; // contains one full packet.
   private int bufRead; //amount of valid data in the buf
   private int bufRead; //amount of valid data in the buf
   private int maxPacketReadLen;
   private int maxPacketReadLen;
-  protected long offsetInBlock;
   protected final String inAddr;
   protected final String inAddr;
   protected final String myAddr;
   protected final String myAddr;
   private String mirrorAddr;
   private String mirrorAddr;
@@ -73,46 +72,83 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
   private Daemon responder = null;
   private Daemon responder = null;
   private BlockTransferThrottler throttler;
   private BlockTransferThrottler throttler;
   private FSDataset.BlockWriteStreams streams;
   private FSDataset.BlockWriteStreams streams;
-  private boolean isRecovery = false;
   private String clientName;
   private String clientName;
   DatanodeInfo srcDataNode = null;
   DatanodeInfo srcDataNode = null;
   private Checksum partialCrc = null;
   private Checksum partialCrc = null;
   private final DataNode datanode;
   private final DataNode datanode;
+  final private ReplicaInPipelineInterface replicaInfo;
 
 
   BlockReceiver(Block block, DataInputStream in, String inAddr,
   BlockReceiver(Block block, DataInputStream in, String inAddr,
-                String myAddr, boolean isRecovery, String clientName, 
-                DatanodeInfo srcDataNode, DataNode datanode) throws IOException {
+                String myAddr, BlockConstructionStage stage, 
+                long newGs, long minBytesRcvd, long maxBytesRcvd, 
+                String clientName, DatanodeInfo srcDataNode, DataNode datanode)
+                throws IOException {
     try{
     try{
       this.block = block;
       this.block = block;
       this.in = in;
       this.in = in;
       this.inAddr = inAddr;
       this.inAddr = inAddr;
       this.myAddr = myAddr;
       this.myAddr = myAddr;
-      this.isRecovery = isRecovery;
       this.clientName = clientName;
       this.clientName = clientName;
-      this.offsetInBlock = 0;
       this.srcDataNode = srcDataNode;
       this.srcDataNode = srcDataNode;
       this.datanode = datanode;
       this.datanode = datanode;
-      this.checksum = DataChecksum.newDataChecksum(in);
-      this.bytesPerChecksum = checksum.getBytesPerChecksum();
-      this.checksumSize = checksum.getChecksumSize();
       //
       //
       // Open local disk out
       // Open local disk out
       //
       //
-      streams = datanode.data.writeToBlock(block, isRecovery);
-      this.finalized = datanode.data.isValidBlock(block);
+      if (clientName.length() == 0) { //replication or move
+        replicaInfo = datanode.data.createTemporary(block);
+      } else {
+        switch (stage) {
+        case PIPELINE_SETUP_CREATE:
+          replicaInfo = datanode.data.createRbw(block);
+          break;
+        case PIPELINE_SETUP_STREAMING_RECOVERY:
+          replicaInfo = datanode.data.recoverRbw(
+              block, newGs, minBytesRcvd, maxBytesRcvd);
+          block.setGenerationStamp(newGs);
+          break;
+        case PIPELINE_SETUP_APPEND:
+          replicaInfo = datanode.data.append(block, newGs, minBytesRcvd);
+          if (datanode.blockScanner != null) { // remove from block scanner
+            datanode.blockScanner.deleteBlock(block);
+          }
+          block.setGenerationStamp(newGs);
+          break;
+        case PIPELINE_SETUP_APPEND_RECOVERY:
+          replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
+          if (datanode.blockScanner != null) { // remove from block scanner
+            datanode.blockScanner.deleteBlock(block);
+          }
+          block.setGenerationStamp(newGs);
+          break;
+        default: throw new IOException("Unsupported stage " + stage + 
+              " while receiving block " + block + " from " + inAddr);
+        }
+      }
+      streams = replicaInfo.createStreams();
       if (streams != null) {
       if (streams != null) {
         this.out = streams.dataOut;
         this.out = streams.dataOut;
         this.checksumOut = new DataOutputStream(new BufferedOutputStream(
         this.checksumOut = new DataOutputStream(new BufferedOutputStream(
                                                   streams.checksumOut, 
                                                   streams.checksumOut, 
                                                   SMALL_BUFFER_SIZE));
                                                   SMALL_BUFFER_SIZE));
-        // If this block is for appends, then remove it from periodic
-        // validation.
-        if (datanode.blockScanner != null && isRecovery) {
-          datanode.blockScanner.deleteBlock(block);
+        
+        // read checksum meta information
+        this.checksum = DataChecksum.newDataChecksum(in);
+        this.bytesPerChecksum = checksum.getBytesPerChecksum();
+        this.checksumSize = checksum.getChecksumSize();
+        
+        // write data chunk header if creating a new replica
+        if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE 
+            || clientName.length() == 0) {
+          BlockMetadataHeader.writeHeader(checksumOut, checksum);
+        } else {
+          datanode.data.setChannelPosition(block, streams, 0, 
+              BlockMetadataHeader.getHeaderSize());
         }
         }
       }
       }
-    } catch (BlockAlreadyExistsException bae) {
+    } catch (ReplicaAlreadyExistsException bae) {
       throw bae;
       throw bae;
+    } catch (ReplicaNotFoundException bne) {
+      throw bne;
     } catch(IOException ioe) {
     } catch(IOException ioe) {
       IOUtils.closeStream(this);
       IOUtils.closeStream(this);
       cleanupBlock();
       cleanupBlock();
@@ -288,7 +324,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
    * It tries to read a full packet with single read call.
    * It tries to read a full packet with single read call.
    * Consecutive packets are usually of the same length.
    * Consecutive packets are usually of the same length.
    */
    */
-  private int readNextPacket() throws IOException {
+  private void readNextPacket() throws IOException {
     /* This dances around buf a little bit, mainly to read 
     /* This dances around buf a little bit, mainly to read 
      * full packet with single read and to accept arbitarary size  
      * full packet with single read and to accept arbitarary size  
      * for next packet at the same time.
      * for next packet at the same time.
@@ -324,12 +360,6 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     int payloadLen = buf.getInt();
     int payloadLen = buf.getInt();
     buf.reset();
     buf.reset();
     
     
-    if (payloadLen == 0) {
-      //end of stream!
-      buf.limit(buf.position() + SIZE_OF_INTEGER);
-      return 0;
-    }
-    
     // check corrupt values for pktLen, 100MB upper limit should be ok?
     // check corrupt values for pktLen, 100MB upper limit should be ok?
     if (payloadLen < 0 || payloadLen > (100*1024*1024)) {
     if (payloadLen < 0 || payloadLen > (100*1024*1024)) {
       throw new IOException("Incorrect value for packet payload : " +
       throw new IOException("Incorrect value for packet payload : " +
@@ -369,42 +399,58 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     if (pktSize > maxPacketReadLen) {
     if (pktSize > maxPacketReadLen) {
       maxPacketReadLen = pktSize;
       maxPacketReadLen = pktSize;
     }
     }
-    
-    return payloadLen;
   }
   }
   
   
   /** 
   /** 
    * Receives and processes a packet. It can contain many chunks.
    * Receives and processes a packet. It can contain many chunks.
-   * returns size of the packet.
+   * returns the number of data bytes that the packet has.
    */
    */
   private int receivePacket() throws IOException {
   private int receivePacket() throws IOException {
-    
-    int payloadLen = readNextPacket();
-    
-    if (payloadLen <= 0) {
-      return payloadLen;
-    }
+    // read the next packet
+    readNextPacket();
     
     
     buf.mark();
     buf.mark();
     //read the header
     //read the header
     buf.getInt(); // packet length
     buf.getInt(); // packet length
-    offsetInBlock = buf.getLong(); // get offset of packet in block
+    long offsetInBlock = buf.getLong(); // get offset of packet in block
+    
+    if (offsetInBlock > replicaInfo.getNumBytes()) {
+      throw new IOException("Received an out-of-sequence packet for " + block + 
+          "from " + inAddr + " at offset " + offsetInBlock +
+          ". Expecting packet starting at " + replicaInfo.getNumBytes());
+    }
     long seqno = buf.getLong();    // get seqno
     long seqno = buf.getLong();    // get seqno
     boolean lastPacketInBlock = (buf.get() != 0);
     boolean lastPacketInBlock = (buf.get() != 0);
     
     
+    int len = buf.getInt();
+    if (len < 0) {
+      throw new IOException("Got wrong length during writeBlock(" + block + 
+                            ") from " + inAddr + " at offset " + 
+                            offsetInBlock + ": " + len); 
+    } 
     int endOfHeader = buf.position();
     int endOfHeader = buf.position();
     buf.reset();
     buf.reset();
     
     
     if (LOG.isDebugEnabled()){
     if (LOG.isDebugEnabled()){
       LOG.debug("Receiving one packet for block " + block +
       LOG.debug("Receiving one packet for block " + block +
-                " of length " + payloadLen +
+                " of length " + len +
                 " seqno " + seqno +
                 " seqno " + seqno +
                 " offsetInBlock " + offsetInBlock +
                 " offsetInBlock " + offsetInBlock +
                 " lastPacketInBlock " + lastPacketInBlock);
                 " lastPacketInBlock " + lastPacketInBlock);
     }
     }
     
     
-    setBlockPosition(offsetInBlock);
+    // update received bytes
+    offsetInBlock += len;
+    if (replicaInfo.getNumBytes() < offsetInBlock) {
+      replicaInfo.setNumBytes(offsetInBlock);
+    }
     
     
+    // put in queue for pending acks
+    if (responder != null) {
+      ((PacketResponder)responder.getRunnable()).enqueue(seqno,
+                                      lastPacketInBlock, offsetInBlock); 
+    }  
+
     //First write the packet to the mirror:
     //First write the packet to the mirror:
     if (mirrorOut != null) {
     if (mirrorOut != null) {
       try {
       try {
@@ -416,19 +462,10 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     }
     }
 
 
     buf.position(endOfHeader);        
     buf.position(endOfHeader);        
-    int len = buf.getInt();
     
     
-    if (len < 0) {
-      throw new IOException("Got wrong length during writeBlock(" + block + 
-                            ") from " + inAddr + " at offset " + 
-                            offsetInBlock + ": " + len); 
-    } 
-
-    if (len == 0) {
-      LOG.debug("Receiving empty packet for block " + block);
+    if (lastPacketInBlock || len == 0) {
+      LOG.debug("Receiving an empty packet or the end of the block " + block);
     } else {
     } else {
-      offsetInBlock += len;
-
       int checksumLen = ((len + bytesPerChecksum - 1)/bytesPerChecksum)*
       int checksumLen = ((len + bytesPerChecksum - 1)/bytesPerChecksum)*
                                                             checksumSize;
                                                             checksumSize;
 
 
@@ -454,8 +491,10 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       }
       }
 
 
       try {
       try {
-        if (!finalized) {
+        if (replicaInfo.getBytesOnDisk()<offsetInBlock) {
           //finally write to the disk :
           //finally write to the disk :
+          setBlockPosition(offsetInBlock-len);
+          
           out.write(pktBuf, dataOff, len);
           out.write(pktBuf, dataOff, len);
 
 
           // If this is a partial chunk, then verify that this is the only
           // If this is a partial chunk, then verify that this is the only
@@ -476,6 +515,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
           } else {
           } else {
             checksumOut.write(pktBuf, checksumOff, checksumLen);
             checksumOut.write(pktBuf, checksumOff, checksumLen);
           }
           }
+          replicaInfo.setBytesOnDisk(offsetInBlock);
           datanode.myMetrics.bytesWritten.inc(len);
           datanode.myMetrics.bytesWritten.inc(len);
         }
         }
       } catch (IOException iex) {
       } catch (IOException iex) {
@@ -487,17 +527,11 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     /// flush entire packet before sending ack
     /// flush entire packet before sending ack
     flush();
     flush();
 
 
-    // put in queue for pending acks
-    if (responder != null) {
-      ((PacketResponder)responder.getRunnable()).enqueue(seqno,
-                                      lastPacketInBlock); 
-    }
-    
     if (throttler != null) { // throttle I/O
     if (throttler != null) { // throttle I/O
-      throttler.throttle(payloadLen);
+      throttler.throttle(len);
     }
     }
     
     
-    return payloadLen;
+    return len;
   }
   }
 
 
   void writeChecksumHeader(DataOutputStream mirrorOut) throws IOException {
   void writeChecksumHeader(DataOutputStream mirrorOut) throws IOException {
@@ -518,10 +552,6 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       throttler = throttlerArg;
       throttler = throttlerArg;
 
 
     try {
     try {
-      // write data chunk header
-      if (!finalized) {
-        BlockMetadataHeader.writeHeader(checksumOut, checksum);
-      }
       if (clientName.length() > 0) {
       if (clientName.length() > 0) {
         responder = new Daemon(datanode.threadGroup, 
         responder = new Daemon(datanode.threadGroup, 
                                new PacketResponder(this, block, mirrIn, 
                                new PacketResponder(this, block, mirrIn, 
@@ -530,20 +560,10 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       }
       }
 
 
       /* 
       /* 
-       * Receive until packet length is zero.
+       * Receive until packet has zero bytes of data.
        */
        */
       while (receivePacket() > 0) {}
       while (receivePacket() > 0) {}
 
 
-      // flush the mirror out
-      if (mirrorOut != null) {
-        try {
-          mirrorOut.writeInt(0); // mark the end of the block
-          mirrorOut.flush();
-        } catch (IOException e) {
-          handleMirrorOutError(e);
-        }
-      }
-
       // wait for all outstanding packet responses. And then
       // wait for all outstanding packet responses. And then
       // indicate responder to gracefully shutdown.
       // indicate responder to gracefully shutdown.
       // Mark that responder has been closed for future processing
       // Mark that responder has been closed for future processing
@@ -560,7 +580,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
         close();
         close();
 
 
         // Finalize the block. Does this fsync()?
         // Finalize the block. Does this fsync()?
-        block.setNumBytes(offsetInBlock);
+        block.setNumBytes(replicaInfo.getNumBytes());
         datanode.data.finalizeBlock(block);
         datanode.data.finalizeBlock(block);
         datanode.myMetrics.blocksWritten.inc();
         datanode.myMetrics.blocksWritten.inc();
       }
       }
@@ -601,21 +621,6 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
    * Sets the file pointer in the local block file to the specified value.
    * Sets the file pointer in the local block file to the specified value.
    */
    */
   private void setBlockPosition(long offsetInBlock) throws IOException {
   private void setBlockPosition(long offsetInBlock) throws IOException {
-    if (finalized) {
-      if (!isRecovery) {
-        throw new IOException("Write to offset " + offsetInBlock +
-                              " of block " + block +
-                              " that is already finalized.");
-      }
-      if (offsetInBlock > datanode.data.getLength(block)) {
-        throw new IOException("Write to offset " + offsetInBlock +
-                              " of block " + block +
-                              " that is already finalized and is of size " +
-                              datanode.data.getLength(block));
-      }
-      return;
-    }
-
     if (datanode.data.getChannelPosition(block, streams) == offsetInBlock) {
     if (datanode.data.getChannelPosition(block, streams) == offsetInBlock) {
       return;                   // nothing to do 
       return;                   // nothing to do 
     }
     }
@@ -732,12 +737,13 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
      * enqueue the seqno that is still be to acked by the downstream datanode.
      * enqueue the seqno that is still be to acked by the downstream datanode.
      * @param seqno
      * @param seqno
      * @param lastPacketInBlock
      * @param lastPacketInBlock
+     * @param lastByteInPacket
      */
      */
-    synchronized void enqueue(long seqno, boolean lastPacketInBlock) {
+    synchronized void enqueue(long seqno, boolean lastPacketInBlock, long lastByteInPacket) {
       if (running) {
       if (running) {
         LOG.debug("PacketResponder " + numTargets + " adding seqno " + seqno +
         LOG.debug("PacketResponder " + numTargets + " adding seqno " + seqno +
                   " to ack queue.");
                   " to ack queue.");
-        ackQueue.addLast(new Packet(seqno, lastPacketInBlock));
+        ackQueue.addLast(new Packet(seqno, lastPacketInBlock, lastByteInPacket));
         notifyAll();
         notifyAll();
       }
       }
     }
     }
@@ -808,26 +814,22 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             // If this is the last packet in block, then close block
             // If this is the last packet in block, then close block
             // file and finalize the block before responding success
             // file and finalize the block before responding success
             if (pkt.lastPacketInBlock) {
             if (pkt.lastPacketInBlock) {
-              if (!receiver.finalized) {
-                receiver.close();
-                final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
-                block.setNumBytes(receiver.offsetInBlock);
-                datanode.data.finalizeBlock(block);
-                datanode.myMetrics.blocksWritten.inc();
-                datanode.notifyNamenodeReceivedBlock(block, 
-                    DataNode.EMPTY_DEL_HINT);
-                if (ClientTraceLog.isInfoEnabled() &&
-                    receiver.clientName.length() > 0) {
-                  long offset = 0;
-                  ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT,
-                        receiver.inAddr, receiver.myAddr, block.getNumBytes(),
-                        "HDFS_WRITE", receiver.clientName, offset,
-                        datanode.dnRegistration.getStorageID(), block, endTime-startTime));
-                } else {
-                  LOG.info("Received block " + block + 
-                           " of size " + block.getNumBytes() + 
-                           " from " + receiver.inAddr);
-                }
+              receiver.close();
+              final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+              block.setNumBytes(replicaInfo.getNumBytes());
+              datanode.data.finalizeBlock(block);
+              datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
+              if (ClientTraceLog.isInfoEnabled() &&
+                  receiver.clientName.length() > 0) {
+                long offset = 0;
+                ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT,
+                    receiver.inAddr, receiver.myAddr, block.getNumBytes(),
+                    "HDFS_WRITE", receiver.clientName, offset,
+                    datanode.dnRegistration.getStorageID(), block, endTime-startTime));
+              } else {
+                LOG.info("Received block " + block + 
+                    " of size " + block.getNumBytes() + 
+                    " from " + receiver.inAddr);
               }
               }
               lastPacket = true;
               lastPacket = true;
             }
             }
@@ -835,6 +837,9 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             replyOut.writeLong(expected);
             replyOut.writeLong(expected);
             SUCCESS.write(replyOut);
             SUCCESS.write(replyOut);
             replyOut.flush();
             replyOut.flush();
+            if (pkt.lastByteInBlock>replicaInfo.getBytesAcked()) {
+              replicaInfo.setBytesAcked(pkt.lastByteInBlock);
+            }
         } catch (Exception e) {
         } catch (Exception e) {
           LOG.warn("IOException in BlockReceiver.lastNodeRun: ", e);
           LOG.warn("IOException in BlockReceiver.lastNodeRun: ", e);
           if (running) {
           if (running) {
@@ -867,6 +872,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       }
       }
 
 
       boolean lastPacketInBlock = false;
       boolean lastPacketInBlock = false;
+      Packet pkt = null;
       final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
       final long startTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
       while (running && datanode.shouldRun && !lastPacketInBlock) {
       while (running && datanode.shouldRun && !lastPacketInBlock) {
 
 
@@ -889,7 +895,6 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
               } else {
               } else {
                 LOG.debug("PacketResponder " + numTargets + " got seqno = " + 
                 LOG.debug("PacketResponder " + numTargets + " got seqno = " + 
                     seqno);
                     seqno);
-                Packet pkt = null;
                 synchronized (this) {
                 synchronized (this) {
                   while (running && datanode.shouldRun && ackQueue.size() == 0) {
                   while (running && datanode.shouldRun && ackQueue.size() == 0) {
                     if (LOG.isDebugEnabled()) {
                     if (LOG.isDebugEnabled()) {
@@ -947,14 +952,12 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             
             
             // If this is the last packet in block, then close block
             // If this is the last packet in block, then close block
             // file and finalize the block before responding success
             // file and finalize the block before responding success
-            if (lastPacketInBlock && !receiver.finalized) {
+            if (lastPacketInBlock) {
               receiver.close();
               receiver.close();
               final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
               final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
-              block.setNumBytes(receiver.offsetInBlock);
+              block.setNumBytes(replicaInfo.getNumBytes());
               datanode.data.finalizeBlock(block);
               datanode.data.finalizeBlock(block);
-              datanode.myMetrics.blocksWritten.inc();
-              datanode.notifyNamenodeReceivedBlock(block, 
-                  DataNode.EMPTY_DEL_HINT);
+              datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
               if (ClientTraceLog.isInfoEnabled() &&
               if (ClientTraceLog.isInfoEnabled() &&
                   receiver.clientName.length() > 0) {
                   receiver.clientName.length() > 0) {
                 long offset = 0;
                 long offset = 0;
@@ -978,12 +981,14 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
                       " responded my status " +
                       " responded my status " +
                       " for seqno " + expected);
                       " for seqno " + expected);
 
 
+            boolean success = true;
             // forward responses from downstream datanodes.
             // forward responses from downstream datanodes.
             for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
             for (int i = 0; i < numTargets && datanode.shouldRun; i++) {
               try {
               try {
                 if (op == SUCCESS) {
                 if (op == SUCCESS) {
                   op = Status.read(mirrorIn);
                   op = Status.read(mirrorIn);
                   if (op != SUCCESS) {
                   if (op != SUCCESS) {
+                    success = false;
                     LOG.debug("PacketResponder for block " + block +
                     LOG.debug("PacketResponder for block " + block +
                               ": error code received from downstream " +
                               ": error code received from downstream " +
                               " datanode[" + i + "] " + op);
                               " datanode[" + i + "] " + op);
@@ -991,6 +996,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
                 }
                 }
               } catch (Throwable e) {
               } catch (Throwable e) {
                 op = ERROR;
                 op = ERROR;
+                success = false;
               }
               }
               op.write(replyOut);
               op.write(replyOut);
             }
             }
@@ -998,6 +1004,10 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
             LOG.debug("PacketResponder " + block + " " + numTargets + 
             LOG.debug("PacketResponder " + block + " " + numTargets + 
                       " responded other status " + " for seqno " + expected);
                       " responded other status " + " for seqno " + expected);
 
 
+            if (pkt != null && success && 
+                pkt.lastByteInBlock>replicaInfo.getBytesAcked()) {
+              replicaInfo.setBytesAcked(pkt.lastByteInBlock);
+            }
             // If we were unable to read the seqno from downstream, then stop.
             // If we were unable to read the seqno from downstream, then stop.
             if (expected == -2) {
             if (expected == -2) {
               running = false;
               running = false;
@@ -1039,10 +1049,12 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
   static private class Packet {
   static private class Packet {
     long seqno;
     long seqno;
     boolean lastPacketInBlock;
     boolean lastPacketInBlock;
+    long lastByteInBlock;
 
 
-    Packet(long seqno, boolean lastPacketInBlock) {
+    Packet(long seqno, boolean lastPacketInBlock, long lastByteInPacket) {
       this.seqno = seqno;
       this.seqno = seqno;
       this.lastPacketInBlock = lastPacketInBlock;
       this.lastPacketInBlock = lastPacketInBlock;
+      this.lastByteInBlock = lastByteInPacket;
     }
     }
   }
   }
 }
 }

+ 46 - 13
src/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -46,13 +46,18 @@ class BlockSender implements java.io.Closeable, FSConstants {
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   static final Log ClientTraceLog = DataNode.ClientTraceLog;
   
   
   private Block block; // the block to read from
   private Block block; // the block to read from
+
+  /** the replica to read from */
+  private final Replica replica;
+  /** The visible length of a replica. */
+  private final long replicaVisibleLength;
+
   private InputStream blockIn; // data stream
   private InputStream blockIn; // data stream
   private long blockInPosition = -1; // updated while using transferTo().
   private long blockInPosition = -1; // updated while using transferTo().
   private DataInputStream checksumIn; // checksum datastream
   private DataInputStream checksumIn; // checksum datastream
   private DataChecksum checksum; // checksum stream
   private DataChecksum checksum; // checksum stream
   private long offset; // starting position to read
   private long offset; // starting position to read
   private long endOffset; // ending position
   private long endOffset; // ending position
-  private long blockLength;
   private int bytesPerChecksum; // chunk size
   private int bytesPerChecksum; // chunk size
   private int checksumSize; // checksum size
   private int checksumSize; // checksum size
   private boolean corruptChecksumOk; // if need to verify checksum
   private boolean corruptChecksumOk; // if need to verify checksum
@@ -86,10 +91,29 @@ class BlockSender implements java.io.Closeable, FSConstants {
       throws IOException {
       throws IOException {
     try {
     try {
       this.block = block;
       this.block = block;
+      synchronized(datanode.data) { 
+        this.replica = datanode.data.getReplica(block.getBlockId());
+        if (replica == null) {
+          throw new ReplicaNotFoundException(block);
+        }
+        this.replicaVisibleLength = replica.getVisibleLength();
+      }
+      if (replica.getGenerationStamp() < block.getGenerationStamp()) {
+        throw new IOException(
+            "replica.getGenerationStamp() < block.getGenerationStamp(), block="
+            + block + ", replica=" + replica);
+      }
+      if (replicaVisibleLength < 0) {
+        throw new IOException("The replica is not readable, block="
+            + block + ", replica=" + replica);
+      }
+      if (DataNode.LOG.isDebugEnabled()) {
+        DataNode.LOG.debug("block=" + block + ", replica=" + replica);
+      }
+      
       this.chunkOffsetOK = chunkOffsetOK;
       this.chunkOffsetOK = chunkOffsetOK;
       this.corruptChecksumOk = corruptChecksumOk;
       this.corruptChecksumOk = corruptChecksumOk;
       this.verifyChecksum = verifyChecksum;
       this.verifyChecksum = verifyChecksum;
-      this.blockLength = datanode.data.getLength(block);
       this.transferToAllowed = datanode.transferToAllowed;
       this.transferToAllowed = datanode.transferToAllowed;
       this.clientTraceFmt = clientTraceFmt;
       this.clientTraceFmt = clientTraceFmt;
 
 
@@ -119,18 +143,18 @@ class BlockSender implements java.io.Closeable, FSConstants {
        * blockLength.
        * blockLength.
        */        
        */        
       bytesPerChecksum = checksum.getBytesPerChecksum();
       bytesPerChecksum = checksum.getBytesPerChecksum();
-      if (bytesPerChecksum > 10*1024*1024 && bytesPerChecksum > blockLength){
+      if (bytesPerChecksum > 10*1024*1024 && bytesPerChecksum > replicaVisibleLength) {
         checksum = DataChecksum.newDataChecksum(checksum.getChecksumType(),
         checksum = DataChecksum.newDataChecksum(checksum.getChecksumType(),
-                                   Math.max((int)blockLength, 10*1024*1024));
+            Math.max((int)replicaVisibleLength, 10*1024*1024));
         bytesPerChecksum = checksum.getBytesPerChecksum();        
         bytesPerChecksum = checksum.getBytesPerChecksum();        
       }
       }
       checksumSize = checksum.getChecksumSize();
       checksumSize = checksum.getChecksumSize();
 
 
       if (length < 0) {
       if (length < 0) {
-        length = blockLength;
+        length = replicaVisibleLength;
       }
       }
 
 
-      endOffset = blockLength;
+      endOffset = replicaVisibleLength;
       if (startOffset < 0 || startOffset > endOffset
       if (startOffset < 0 || startOffset > endOffset
           || (length + startOffset) > endOffset) {
           || (length + startOffset) > endOffset) {
         String msg = " Offset " + startOffset + " and length " + length
         String msg = " Offset " + startOffset + " and length " + length
@@ -163,6 +187,18 @@ class BlockSender implements java.io.Closeable, FSConstants {
       }
       }
       seqno = 0;
       seqno = 0;
 
 
+      //sleep a few times if getBytesOnDisk() < visible length
+      for(int i = 0; i < 30 && replica.getBytesOnDisk() < replicaVisibleLength; i++) {
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException ie) {
+          throw new IOException(ie);
+        }
+      }
+      if (DataNode.LOG.isDebugEnabled()) {
+        DataNode.LOG.debug("replica=" + replica);
+      }
+
       blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
       blockIn = datanode.data.getBlockInputStream(block, offset); // seek to offset
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       IOUtils.closeStream(this);
       IOUtils.closeStream(this);
@@ -234,10 +270,6 @@ class BlockSender implements java.io.Closeable, FSConstants {
 
 
     int len = Math.min((int) (endOffset - offset),
     int len = Math.min((int) (endOffset - offset),
                        bytesPerChecksum*maxChunks);
                        bytesPerChecksum*maxChunks);
-    if (len == 0) {
-      return 0;
-    }
-
     int numChunks = (len + bytesPerChecksum - 1)/bytesPerChecksum;
     int numChunks = (len + bytesPerChecksum - 1)/bytesPerChecksum;
     int packetLen = len + numChunks*checksumSize + 4;
     int packetLen = len + numChunks*checksumSize + 4;
     pkt.clear();
     pkt.clear();
@@ -246,7 +278,7 @@ class BlockSender implements java.io.Closeable, FSConstants {
     pkt.putInt(packetLen);
     pkt.putInt(packetLen);
     pkt.putLong(offset);
     pkt.putLong(offset);
     pkt.putLong(seqno);
     pkt.putLong(seqno);
-    pkt.put((byte)((offset + len >= endOffset) ? 1 : 0));
+    pkt.put((byte)((len == 0) ? 1 : 0));
                //why no ByteBuf.putBoolean()?
                //why no ByteBuf.putBoolean()?
     pkt.putInt(len);
     pkt.putInt(len);
     
     
@@ -407,7 +439,8 @@ class BlockSender implements java.io.Closeable, FSConstants {
         seqno++;
         seqno++;
       }
       }
       try {
       try {
-        out.writeInt(0); // mark the end of block        
+        // send an empty packet to mark the end of the block
+        sendChunks(pktBuf, maxChunksPerPacket, streamForSendChunks);        
         out.flush();
         out.flush();
       } catch (IOException e) { //socket error
       } catch (IOException e) { //socket error
         throw ioeToSocketException(e);
         throw ioeToSocketException(e);
@@ -420,7 +453,7 @@ class BlockSender implements java.io.Closeable, FSConstants {
       close();
       close();
     }
     }
 
 
-    blockReadFully = (initialOffset == 0 && offset >= blockLength);
+    blockReadFully = initialOffset == 0 && offset >= replicaVisibleLength;
 
 
     return totalRead;
     return totalRead;
   }
   }

+ 3 - 3
src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -29,11 +29,11 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.text.DateFormat;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Date;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Random;
 import java.util.Random;
 import java.util.TreeSet;
 import java.util.TreeSet;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
@@ -211,8 +211,8 @@ class DataBlockScanner implements Runnable {
   private void init() {
   private void init() {
     
     
     // get the list of blocks and arrange them in random order
     // get the list of blocks and arrange them in random order
-    Block arr[] = dataset.getBlockReport();
-    Collections.shuffle(Arrays.asList(arr));
+    List<Block> arr = dataset.getFinalizedBlocks();
+    Collections.shuffle(arr);
     
     
     blockInfoSet = new TreeSet<BlockScanInfo>();
     blockInfoSet = new TreeSet<BlockScanInfo>();
     blockMap = new HashMap<Block, BlockScanInfo>();
     blockMap = new HashMap<Block, BlockScanInfo>();

+ 192 - 130
src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -34,11 +34,10 @@ import java.security.SecureRandom;
 import java.util.AbstractList;
 import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 import java.util.Random;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicInteger;
 
 
@@ -55,11 +54,13 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -68,6 +69,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.namenode.StreamFile;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -75,7 +77,9 @@ import org.apache.hadoop.hdfs.server.protocol.DisallowedDatanodeException;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
@@ -168,8 +172,6 @@ public class DataNode extends Configured
 
 
   volatile boolean shouldRun = true;
   volatile boolean shouldRun = true;
   private LinkedList<Block> receivedBlockList = new LinkedList<Block>();
   private LinkedList<Block> receivedBlockList = new LinkedList<Block>();
-  /** list of blocks being recovered */
-  private final Map<Block, Block> ongoingRecovery = new HashMap<Block, Block>();
   private LinkedList<String> delHints = new LinkedList<String>();
   private LinkedList<String> delHints = new LinkedList<String>();
   public final static String EMPTY_DEL_HINT = "";
   public final static String EMPTY_DEL_HINT = "";
   AtomicInteger xmitsInProgress = new AtomicInteger();
   AtomicInteger xmitsInProgress = new AtomicInteger();
@@ -912,7 +914,7 @@ public class DataNode extends Configured
       processDistributedUpgradeCommand((UpgradeCommand)cmd);
       processDistributedUpgradeCommand((UpgradeCommand)cmd);
       break;
       break;
     case DatanodeProtocol.DNA_RECOVERBLOCK:
     case DatanodeProtocol.DNA_RECOVERBLOCK:
-      recoverBlocks(bcmd.getBlocks(), bcmd.getTargets());
+      recoverBlocks(((BlockRecoveryCommand)cmd).getRecoveringBlocks());
       break;
       break;
     case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
     case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
       LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
       LOG.info("DatanodeCommand action: DNA_ACCESSKEYUPDATE");
@@ -989,13 +991,12 @@ public class DataNode extends Configured
       // and can be safely GC'ed.
       // and can be safely GC'ed.
       //
       //
       long brStartTime = now();
       long brStartTime = now();
-      Block[] bReport = data.getBlockReport();
+      BlockListAsLongs bReport = data.getBlockReport();
 
 
-      cmd = namenode.blockReport(dnRegistration,
-              BlockListAsLongs.convertToArrayLongs(bReport));
+      cmd = namenode.blockReport(dnRegistration, bReport.getBlockListAsLongs());
       long brTime = now() - brStartTime;
       long brTime = now() - brStartTime;
       myMetrics.blockReports.inc(brTime);
       myMetrics.blockReports.inc(brTime);
-      LOG.info("BlockReport of " + bReport.length +
+      LOG.info("BlockReport of " + bReport.getNumberOfBlocks() +
           " blocks got processed in " + brTime + " msecs");
           " blocks got processed in " + brTime + " msecs");
       //
       //
       // If we have sent the first block report, then wait a random
       // If we have sent the first block report, then wait a random
@@ -1250,7 +1251,8 @@ public class DataNode extends Configured
               EnumSet.of(AccessTokenHandler.AccessMode.WRITE));
               EnumSet.of(AccessTokenHandler.AccessMode.WRITE));
         }
         }
         DataTransferProtocol.Sender.opWriteBlock(out,
         DataTransferProtocol.Sender.opWriteBlock(out,
-            b.getBlockId(), b.getGenerationStamp(), 0, false, "",
+            b.getBlockId(), b.getGenerationStamp(), 0, 
+            BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, "",
             srcNode, targets, accessToken);
             srcNode, targets, accessToken);
 
 
         // send data & checksum
         // send data & checksum
@@ -1273,6 +1275,20 @@ public class DataNode extends Configured
       }
       }
     }
     }
   }
   }
+  
+  /**
+   * After a block becomes finalized, a datanode increases metric counter,
+   * notifies namenode, and adds it to the block scanner
+   * @param block
+   * @param delHint
+   */
+  void closeBlock(Block block, String delHint) {
+    myMetrics.blocksWritten.inc();
+    notifyNamenodeReceivedBlock(block, delHint);
+    if (blockScanner != null) {
+      blockScanner.addBlock(block);
+    }
+  }
 
 
   /**
   /**
    * No matter what kind of exception we get, keep retrying to offerService().
    * No matter what kind of exception we get, keep retrying to offerService().
@@ -1514,16 +1530,16 @@ public class DataNode extends Configured
     return info;
     return info;
   }
   }
 
 
-  public Daemon recoverBlocks(final Block[] blocks, final DatanodeInfo[][] targets) {
+  public Daemon recoverBlocks(final Collection<RecoveringBlock> blocks) {
     Daemon d = new Daemon(threadGroup, new Runnable() {
     Daemon d = new Daemon(threadGroup, new Runnable() {
       /** Recover a list of blocks. It is run by the primary datanode. */
       /** Recover a list of blocks. It is run by the primary datanode. */
       public void run() {
       public void run() {
-        for(int i = 0; i < blocks.length; i++) {
+        for(RecoveringBlock b : blocks) {
           try {
           try {
-            logRecoverBlock("NameNode", blocks[i], targets[i]);
-            recoverBlock(blocks[i], false, targets[i], true);
+            logRecoverBlock("NameNode", b.getBlock(), b.getLocations());
+            recoverBlock(b);
           } catch (IOException e) {
           } catch (IOException e) {
-            LOG.warn("recoverBlocks FAILED, blocks[" + i + "]=" + blocks[i], e);
+            LOG.warn("recoverBlocks FAILED: " + b, e);
           }
           }
         }
         }
       }
       }
@@ -1548,6 +1564,38 @@ public class DataNode extends Configured
     }
     }
   }
   }
 
 
+  @Override // InterDatanodeProtocol
+  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
+  throws IOException {
+    return data.initReplicaRecovery(rBlock);
+  }
+
+  /**
+   * Convenience method, which unwraps RemoteException.
+   * @throws IOException not a RemoteException.
+   */
+  private static ReplicaRecoveryInfo callInitReplicaRecovery(
+      InterDatanodeProtocol datanode,
+      RecoveringBlock rBlock) throws IOException {
+    try {
+      return datanode.initReplicaRecovery(rBlock);
+    } catch(RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
+  }
+
+  /**
+   * Update replica with the new generation stamp and length.  
+   */
+  @Override // InterDatanodeProtocol
+  public Block updateReplicaUnderRecovery(Block oldBlock,
+                                          long recoveryId,
+                                          long newLength) throws IOException {
+    ReplicaInfo r =
+      data.updateReplicaUnderRecovery(oldBlock, recoveryId, newLength);
+    return new Block(r);
+  }
+
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   public long getProtocolVersion(String protocol, long clientVersion
   public long getProtocolVersion(String protocol, long clientVersion
       ) throws IOException {
       ) throws IOException {
@@ -1560,164 +1608,171 @@ public class DataNode extends Configured
         + ": " + protocol);
         + ": " + protocol);
   }
   }
 
 
-  /** A convenient class used in lease recovery */
+  /** A convenient class used in block recovery */
   private static class BlockRecord { 
   private static class BlockRecord { 
     final DatanodeID id;
     final DatanodeID id;
     final InterDatanodeProtocol datanode;
     final InterDatanodeProtocol datanode;
-    final Block block;
+    final ReplicaRecoveryInfo rInfo;
     
     
-    BlockRecord(DatanodeID id, InterDatanodeProtocol datanode, Block block) {
+    BlockRecord(DatanodeID id,
+                InterDatanodeProtocol datanode,
+                ReplicaRecoveryInfo rInfo) {
       this.id = id;
       this.id = id;
       this.datanode = datanode;
       this.datanode = datanode;
-      this.block = block;
+      this.rInfo = rInfo;
     }
     }
 
 
     /** {@inheritDoc} */
     /** {@inheritDoc} */
     public String toString() {
     public String toString() {
-      return "block:" + block + " node:" + id;
+      return "block:" + rInfo + " node:" + id;
     }
     }
   }
   }
 
 
   /** Recover a block */
   /** Recover a block */
-  private LocatedBlock recoverBlock(Block block, boolean keepLength,
-      DatanodeInfo[] targets, boolean closeFile) throws IOException {
-
+  private void recoverBlock(RecoveringBlock rBlock) throws IOException {
+    Block block = rBlock.getBlock();
+    DatanodeInfo[] targets = rBlock.getLocations();
     DatanodeID[] datanodeids = (DatanodeID[])targets;
     DatanodeID[] datanodeids = (DatanodeID[])targets;
-    // If the block is already being recovered, then skip recovering it.
-    // This can happen if the namenode and client start recovering the same
-    // file at the same time.
-    synchronized (ongoingRecovery) {
-      Block tmp = new Block();
-      tmp.set(block.getBlockId(), block.getNumBytes(), GenerationStamp.WILDCARD_STAMP);
-      if (ongoingRecovery.get(tmp) != null) {
-        String msg = "Block " + block + " is already being recovered, " +
-                     " ignoring this request to recover it.";
-        LOG.info(msg);
-        throw new IOException(msg);
-      }
-      ongoingRecovery.put(block, block);
-    }
-    try {
-      List<BlockRecord> syncList = new ArrayList<BlockRecord>();
-      long minlength = Long.MAX_VALUE;
-      int errorCount = 0;
+    List<BlockRecord> syncList = new ArrayList<BlockRecord>(datanodeids.length);
+    int errorCount = 0;
 
 
-      //check generation stamps
-      for(DatanodeID id : datanodeids) {
-        try {
-          InterDatanodeProtocol datanode = dnRegistration.equals(id)?
-              this: DataNode.createInterDataNodeProtocolProxy(id, getConf());
-          BlockMetaDataInfo info = datanode.getBlockMetaDataInfo(block);
-          if (info != null && info.getGenerationStamp() >= block.getGenerationStamp()) {
-            if (keepLength) {
-              if (info.getNumBytes() == block.getNumBytes()) {
-                syncList.add(new BlockRecord(id, datanode, new Block(info)));
-              }
-            }
-            else {
-              syncList.add(new BlockRecord(id, datanode, new Block(info)));
-              if (info.getNumBytes() < minlength) {
-                minlength = info.getNumBytes();
-              }
-            }
-          }
-        } catch (IOException e) {
-          ++errorCount;
-          InterDatanodeProtocol.LOG.warn(
-              "Failed to getBlockMetaDataInfo for block (=" + block 
-              + ") from datanode (=" + id + ")", e);
+    //check generation stamps
+    for(DatanodeID id : datanodeids) {
+      try {
+        InterDatanodeProtocol datanode = dnRegistration.equals(id)?
+            this: DataNode.createInterDataNodeProtocolProxy(id, getConf());
+        ReplicaRecoveryInfo info = callInitReplicaRecovery(datanode, rBlock);
+        if (info != null &&
+            info.getGenerationStamp() >= block.getGenerationStamp() &&
+            info.getNumBytes() > 0) {
+          syncList.add(new BlockRecord(id, datanode, info));
         }
         }
+      } catch (RecoveryInProgressException ripE) {
+        InterDatanodeProtocol.LOG.warn(
+            "Recovery for replica " + block + " on data-node " + id
+            + " is already in progress. Recovery id = "
+            + rBlock.getNewGenerationStamp() + " is aborted.", ripE);
+        return;
+      } catch (IOException e) {
+        ++errorCount;
+        InterDatanodeProtocol.LOG.warn(
+            "Failed to obtain replica info for block (=" + block 
+            + ") from datanode (=" + id + ")", e);
       }
       }
+    }
 
 
-      if (syncList.isEmpty() && errorCount > 0) {
-        throw new IOException("All datanodes failed: block=" + block
-            + ", datanodeids=" + Arrays.asList(datanodeids));
-      }
-      if (!keepLength) {
-        block.setNumBytes(minlength);
-      }
-      return syncBlock(block, syncList, targets, closeFile);
-    } finally {
-      synchronized (ongoingRecovery) {
-        ongoingRecovery.remove(block);
-      }
+    if (errorCount == datanodeids.length) {
+      throw new IOException("All datanodes failed: block=" + block
+          + ", datanodeids=" + Arrays.asList(datanodeids));
     }
     }
+
+    syncBlock(rBlock, syncList);
   }
   }
 
 
   /** Block synchronization */
   /** Block synchronization */
-  private LocatedBlock syncBlock(Block block, List<BlockRecord> syncList,
-      DatanodeInfo[] targets, boolean closeFile) throws IOException {
+  private void syncBlock(RecoveringBlock rBlock,
+                         List<BlockRecord> syncList) throws IOException {
+    Block block = rBlock.getBlock();
+    long recoveryId = rBlock.getNewGenerationStamp();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("block=" + block + ", (length=" + block.getNumBytes()
       LOG.debug("block=" + block + ", (length=" + block.getNumBytes()
-          + "), syncList=" + syncList + ", closeFile=" + closeFile);
+          + "), syncList=" + syncList);
     }
     }
 
 
-    //syncList.isEmpty() that all datanodes do not have the block
-    //so the block can be deleted.
+    // syncList.isEmpty() means that all data-nodes do not have the block
+    // or their replicas have 0 length.
+    // The block can be deleted.
     if (syncList.isEmpty()) {
     if (syncList.isEmpty()) {
-      namenode.commitBlockSynchronization(block, 0, 0, closeFile, true,
-          DatanodeID.EMPTY_ARRAY);
-      //always return a new access token even if everything else stays the same
-      LocatedBlock b = new LocatedBlock(block, targets);
-      if (isAccessTokenEnabled) {
-        b.setAccessToken(accessTokenHandler.generateToken(null, b.getBlock()
-            .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
-      }
-      return b;
+      namenode.commitBlockSynchronization(block, recoveryId, 0,
+          true, true, DatanodeID.EMPTY_ARRAY);
+      return;
     }
     }
 
 
-    List<DatanodeID> successList = new ArrayList<DatanodeID>();
+    // Calculate the best available replica state.
+    ReplicaState bestState = ReplicaState.RWR;
+    long finalizedLength = -1;
+    for(BlockRecord r : syncList) {
+      assert r.rInfo.getNumBytes() > 0 : "zero length replica";
+      ReplicaState rState = r.rInfo.getOriginalReplicaState(); 
+      if(rState.getValue() < bestState.getValue())
+        bestState = rState;
+      if(rState == ReplicaState.FINALIZED) {
+        if(finalizedLength > 0 && finalizedLength != r.rInfo.getNumBytes())
+          throw new IOException("Inconsistent size of finalized replicas. " +
+              "Replica " + r.rInfo + " expected size: " + finalizedLength);
+        finalizedLength = r.rInfo.getNumBytes();
+      }
+    }
 
 
-    long generationstamp = namenode.nextGenerationStamp(block);
-    Block newblock = new Block(block.getBlockId(), block.getNumBytes(), generationstamp);
+    // Calculate list of nodes that will participate in the recovery
+    // and the new block size
+    List<BlockRecord> participatingList = new ArrayList<BlockRecord>();
+    Block newBlock = new Block(block.getBlockId(), -1, recoveryId);
+    switch(bestState) {
+    case FINALIZED:
+      assert finalizedLength > 0 : "finalizedLength is not positive";
+      for(BlockRecord r : syncList) {
+        ReplicaState rState = r.rInfo.getOriginalReplicaState();
+        if(rState == ReplicaState.FINALIZED ||
+           rState == ReplicaState.RBW &&
+                      r.rInfo.getNumBytes() == finalizedLength)
+          participatingList.add(r);
+      }
+      newBlock.setNumBytes(finalizedLength);
+      break;
+    case RBW:
+    case RWR:
+      long minLength = Long.MAX_VALUE;
+      for(BlockRecord r : syncList) {
+        ReplicaState rState = r.rInfo.getOriginalReplicaState();
+        if(rState == bestState) {
+          minLength = Math.min(minLength, r.rInfo.getNumBytes());
+          participatingList.add(r);
+        }
+      }
+      newBlock.setNumBytes(minLength);
+      break;
+    case RUR:
+    case TEMPORARY:
+      assert false : "bad replica state: " + bestState;
+    }
 
 
-    for(BlockRecord r : syncList) {
+    List<DatanodeID> failedList = new ArrayList<DatanodeID>();
+    List<DatanodeID> successList = new ArrayList<DatanodeID>();
+    for(BlockRecord r : participatingList) {
       try {
       try {
-        r.datanode.updateBlock(r.block, newblock, closeFile);
+        Block reply = r.datanode.updateReplicaUnderRecovery(
+            r.rInfo, recoveryId, newBlock.getNumBytes());
+        assert reply.equals(newBlock) &&
+               reply.getNumBytes() == newBlock.getNumBytes() :
+          "Updated replica must be the same as the new block.";
         successList.add(r.id);
         successList.add(r.id);
       } catch (IOException e) {
       } catch (IOException e) {
         InterDatanodeProtocol.LOG.warn("Failed to updateBlock (newblock="
         InterDatanodeProtocol.LOG.warn("Failed to updateBlock (newblock="
-            + newblock + ", datanode=" + r.id + ")", e);
+            + newBlock + ", datanode=" + r.id + ")", e);
+        failedList.add(r.id);
       }
       }
     }
     }
 
 
-    if (!successList.isEmpty()) {
-      DatanodeID[] nlist = successList.toArray(new DatanodeID[successList.size()]);
-
-      namenode.commitBlockSynchronization(block,
-          newblock.getGenerationStamp(), newblock.getNumBytes(), closeFile, false,
-          nlist);
-      DatanodeInfo[] info = new DatanodeInfo[nlist.length];
-      for (int i = 0; i < nlist.length; i++) {
-        info[i] = new DatanodeInfo(nlist[i]);
-      }
-      LocatedBlock b = new LocatedBlock(newblock, info); // success
-      // should have used client ID to generate access token, but since 
-      // owner ID is not checked, we simply pass null for now.
-      if (isAccessTokenEnabled) {
-        b.setAccessToken(accessTokenHandler.generateToken(null, b.getBlock()
-            .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
+    // If any of the data-nodes failed, the recovery fails, because
+    // we never know the actual state of the replica on failed data-nodes.
+    // The recovery should be started over.
+    if(!failedList.isEmpty()) {
+      StringBuilder b = new StringBuilder();
+      for(DatanodeID id : failedList) {
+        b.append("\n  " + id);
       }
       }
-      return b;
+      throw new IOException("Cannot recover " + block + ", the following "
+          + failedList.size() + " data-nodes failed {" + b + "\n}");
     }
     }
 
 
-    //failed
-    StringBuilder b = new StringBuilder();
-    for(BlockRecord r : syncList) {
-      b.append("\n  " + r.id);
-    }
-    throw new IOException("Cannot recover " + block + ", none of these "
-        + syncList.size() + " datanodes success {" + b + "\n}");
+    // Notify the name-node about successfully recovered replicas.
+    DatanodeID[] nlist = successList.toArray(new DatanodeID[successList.size()]);
+    namenode.commitBlockSynchronization(block,
+        newBlock.getGenerationStamp(), newBlock.getNumBytes(), true, false,
+        nlist);
   }
   }
   
   
-  // ClientDataNodeProtocol implementation
-  /** {@inheritDoc} */
-  public LocatedBlock recoverBlock(Block block, boolean keepLength, DatanodeInfo[] targets
-      ) throws IOException {
-    logRecoverBlock("Client", block, targets);
-    return recoverBlock(block, keepLength, targets, false);
-  }
-
   private static void logRecoverBlock(String who,
   private static void logRecoverBlock(String who,
       Block block, DatanodeID[] targets) {
       Block block, DatanodeID[] targets) {
     StringBuilder msg = new StringBuilder(targets[0].getName());
     StringBuilder msg = new StringBuilder(targets[0].getName());
@@ -1727,4 +1782,11 @@ public class DataNode extends Configured
     LOG.info(who + " calls recoverBlock(block=" + block
     LOG.info(who + " calls recoverBlock(block=" + block
         + ", targets=[" + msg + "])");
         + ", targets=[" + msg + "])");
   }
   }
+
+  // ClientDataNodeProtocol implementation
+  /** {@inheritDoc} */
+  @Override // ClientDataNodeProtocol
+  public long getReplicaVisibleLength(final Block block) throws IOException {
+    return data.getReplicaVisibleLength(block);
+  }
 }
 }

+ 61 - 3
src/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -53,6 +53,9 @@ public class DataStorage extends Storage {
   final static String BLOCK_SUBDIR_PREFIX = "subdir";
   final static String BLOCK_SUBDIR_PREFIX = "subdir";
   final static String BLOCK_FILE_PREFIX = "blk_";
   final static String BLOCK_FILE_PREFIX = "blk_";
   final static String COPY_FILE_PREFIX = "dncp_";
   final static String COPY_FILE_PREFIX = "dncp_";
+  final static String STORAGE_DIR_RBW = "rbw";
+  final static String STORAGE_DIR_FINALIZED = "finalized";
+  final static String STORAGE_DIR_DETACHED = "detach";
   
   
   private String storageID;
   private String storageID;
 
 
@@ -270,6 +273,8 @@ public class DataStorage extends Storage {
     File curDir = sd.getCurrentDir();
     File curDir = sd.getCurrentDir();
     File prevDir = sd.getPreviousDir();
     File prevDir = sd.getPreviousDir();
     assert curDir.exists() : "Current directory must exist.";
     assert curDir.exists() : "Current directory must exist.";
+    // Cleanup directory "detach"
+    cleanupDetachDir(new File(curDir, STORAGE_DIR_DETACHED));
     // delete previous dir before upgrading
     // delete previous dir before upgrading
     if (prevDir.exists())
     if (prevDir.exists())
       deleteDir(prevDir);
       deleteDir(prevDir);
@@ -277,8 +282,11 @@ public class DataStorage extends Storage {
     assert !tmpDir.exists() : "previous.tmp directory must not exist.";
     assert !tmpDir.exists() : "previous.tmp directory must not exist.";
     // rename current to tmp
     // rename current to tmp
     rename(curDir, tmpDir);
     rename(curDir, tmpDir);
-    // hardlink blocks
-    linkBlocks(tmpDir, curDir, this.getLayoutVersion());
+    // hard link finalized & rbw blocks
+    linkAllBlocks(tmpDir, curDir);
+    // create current directory if not exists
+    if (!curDir.exists() && !curDir.mkdirs())
+      throw new IOException("Cannot create directory " + curDir);
     // write version file
     // write version file
     this.layoutVersion = FSConstants.LAYOUT_VERSION;
     this.layoutVersion = FSConstants.LAYOUT_VERSION;
     assert this.namespaceID == nsInfo.getNamespaceID() :
     assert this.namespaceID == nsInfo.getNamespaceID() :
@@ -290,6 +298,30 @@ public class DataStorage extends Storage {
     LOG.info("Upgrade of " + sd.getRoot()+ " is complete.");
     LOG.info("Upgrade of " + sd.getRoot()+ " is complete.");
   }
   }
 
 
+  /**
+   * Cleanup the detachDir. 
+   * 
+   * If the directory is not empty report an error; 
+   * Otherwise remove the directory.
+   * 
+   * @param detachDir detach directory
+   * @throws IOException if the directory is not empty or it can not be removed
+   */
+  private void cleanupDetachDir(File detachDir) throws IOException {
+    if (layoutVersion >= PRE_RBW_LAYOUT_VERSION &&
+        detachDir.exists() && detachDir.isDirectory() ) {
+      
+        if (detachDir.list().length != 0 ) {
+          throw new IOException("Detached directory " + detachDir +
+              " is not empty. Please manually move each file under this " +
+              "directory to the finalized directory if the finalized " +
+              "directory tree does not have the file.");
+        } else if (!detachDir.delete()) {
+          throw new IOException("Cannot remove directory " + detachDir);
+        }
+    }
+  }
+  
   void doRollback( StorageDirectory sd,
   void doRollback( StorageDirectory sd,
                    NamespaceInfo nsInfo
                    NamespaceInfo nsInfo
                    ) throws IOException {
                    ) throws IOException {
@@ -359,8 +391,34 @@ public class DataStorage extends Storage {
       doFinalize(it.next());
       doFinalize(it.next());
     }
     }
   }
   }
+
+  /**
+   * Hardlink all finalized and RBW blocks in fromDir to toDir
+   * @param fromDir directory where the snapshot is stored
+   * @param toDir the current data directory
+   * @throws IOException if error occurs during hardlink
+   */
+  private void linkAllBlocks(File fromDir, File toDir) throws IOException {
+    // do the link
+    int diskLayoutVersion = this.getLayoutVersion();
+    if (diskLayoutVersion < PRE_RBW_LAYOUT_VERSION) { // RBW version
+      // hardlink finalized blocks in tmpDir/finalized
+      linkBlocks(new File(fromDir, STORAGE_DIR_FINALIZED), 
+          new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion);
+      // hardlink rbw blocks in tmpDir/finalized
+      linkBlocks(new File(fromDir, STORAGE_DIR_RBW), 
+          new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion);
+    } else { // pre-RBW version
+      // hardlink finalized blocks in tmpDir
+      linkBlocks(fromDir, 
+          new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion);      
+    }    
+  }
   
   
   static void linkBlocks(File from, File to, int oldLV) throws IOException {
   static void linkBlocks(File from, File to, int oldLV) throws IOException {
+    if (!from.exists()) {
+      return;
+    }
     if (!from.isDirectory()) {
     if (!from.isDirectory()) {
       if (from.getName().startsWith(COPY_FILE_PREFIX)) {
       if (from.getName().startsWith(COPY_FILE_PREFIX)) {
         FileInputStream in = new FileInputStream(from);
         FileInputStream in = new FileInputStream(from);
@@ -387,7 +445,7 @@ public class DataStorage extends Storage {
       return;
       return;
     }
     }
     // from is a directory
     // from is a directory
-    if (!to.mkdir())
+    if (!to.mkdirs())
       throw new IOException("Cannot create directory " + to);
       throw new IOException("Cannot create directory " + to);
     String[] blockNames = from.list(new java.io.FilenameFilter() {
     String[] blockNames = from.list(new java.io.FilenameFilter() {
         public boolean accept(File dir, String name) {
         public boolean accept(File dir, String name) {

+ 38 - 20
src/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputStream;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.MetaDataInputStream;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -208,7 +209,8 @@ class DataXceiver extends DataTransferProtocol.Receiver
    */
    */
   @Override
   @Override
   protected void opWriteBlock(DataInputStream in, long blockId, long blockGs,
   protected void opWriteBlock(DataInputStream in, long blockId, long blockGs,
-      int pipelineSize, boolean isRecovery,
+      int pipelineSize, BlockConstructionStage stage,
+      long newGs, long minBytesRcvd, long maxBytesRcvd,
       String client, DatanodeInfo srcDataNode, DatanodeInfo[] targets,
       String client, DatanodeInfo srcDataNode, DatanodeInfo[] targets,
       AccessToken accessToken) throws IOException {
       AccessToken accessToken) throws IOException {
 
 
@@ -250,11 +252,17 @@ class DataXceiver extends DataTransferProtocol.Receiver
     String firstBadLink = "";           // first datanode that failed in connection setup
     String firstBadLink = "";           // first datanode that failed in connection setup
     DataTransferProtocol.Status mirrorInStatus = SUCCESS;
     DataTransferProtocol.Status mirrorInStatus = SUCCESS;
     try {
     try {
-      // open a block receiver and check if the block does not exist
-      blockReceiver = new BlockReceiver(block, in, 
-          s.getRemoteSocketAddress().toString(),
-          s.getLocalSocketAddress().toString(),
-          isRecovery, client, srcDataNode, datanode);
+      if (client.length() == 0 || 
+          stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+        // open a block receiver
+        blockReceiver = new BlockReceiver(block, in, 
+            s.getRemoteSocketAddress().toString(),
+            s.getLocalSocketAddress().toString(),
+            stage, newGs, minBytesRcvd, maxBytesRcvd,
+            client, srcDataNode, datanode);
+      } else {
+        datanode.data.recoverClose(block, newGs, minBytesRcvd);
+      }
 
 
       //
       //
       // Open network conn to backup machine, if 
       // Open network conn to backup machine, if 
@@ -282,10 +290,13 @@ class DataXceiver extends DataTransferProtocol.Receiver
 
 
           // Write header: Copied from DFSClient.java!
           // Write header: Copied from DFSClient.java!
           DataTransferProtocol.Sender.opWriteBlock(mirrorOut,
           DataTransferProtocol.Sender.opWriteBlock(mirrorOut,
-              block.getBlockId(), block.getGenerationStamp(), pipelineSize,
-              isRecovery, client, srcDataNode, targets, accessToken);
+              blockId, blockGs, 
+              pipelineSize, stage, newGs, minBytesRcvd, maxBytesRcvd, client, 
+              srcDataNode, targets, accessToken);
 
 
-          blockReceiver.writeChecksumHeader(mirrorOut);
+          if (blockReceiver != null) { // send checksum header
+            blockReceiver.writeChecksumHeader(mirrorOut);
+          }
           mirrorOut.flush();
           mirrorOut.flush();
 
 
           // read connect ack (only for clients, not for replication req)
           // read connect ack (only for clients, not for replication req)
@@ -336,24 +347,31 @@ class DataXceiver extends DataTransferProtocol.Receiver
       }
       }
 
 
       // receive the block and mirror to the next target
       // receive the block and mirror to the next target
-      String mirrorAddr = (mirrorSock == null) ? null : mirrorNode;
-      blockReceiver.receiveBlock(mirrorOut, mirrorIn, replyOut,
-                                 mirrorAddr, null, targets.length);
+      if (blockReceiver != null) {
+        String mirrorAddr = (mirrorSock == null) ? null : mirrorNode;
+        blockReceiver.receiveBlock(mirrorOut, mirrorIn, replyOut,
+            mirrorAddr, null, targets.length);
+      }
 
 
-      // if this write is for a replication request (and not
-      // from a client), then confirm block. For client-writes,
+      // update its generation stamp
+      if (client.length() != 0 && 
+          stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+        block.setGenerationStamp(newGs);
+        block.setNumBytes(minBytesRcvd);
+      }
+      
+      // if this write is for a replication request or recovering
+      // a failed close for client, then confirm block. For other client-writes,
       // the block is finalized in the PacketResponder.
       // the block is finalized in the PacketResponder.
-      if (client.length() == 0) {
-        datanode.notifyNamenodeReceivedBlock(block, DataNode.EMPTY_DEL_HINT);
+      if (client.length() == 0 || 
+          stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+        datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
         LOG.info("Received block " + block + 
         LOG.info("Received block " + block + 
                  " src: " + remoteAddress +
                  " src: " + remoteAddress +
                  " dest: " + localAddress +
                  " dest: " + localAddress +
                  " of size " + block.getNumBytes());
                  " of size " + block.getNumBytes());
       }
       }
 
 
-      if (datanode.blockScanner != null) {
-        datanode.blockScanner.addBlock(block);
-      }
       
       
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       LOG.info("writeBlock " + block + " received exception " + ioe);
       LOG.info("writeBlock " + block + " received exception " + ioe);
@@ -569,7 +587,7 @@ class DataXceiver extends DataTransferProtocol.Receiver
       blockReceiver = new BlockReceiver(
       blockReceiver = new BlockReceiver(
           block, proxyReply, proxySock.getRemoteSocketAddress().toString(),
           block, proxyReply, proxySock.getRemoteSocketAddress().toString(),
           proxySock.getLocalSocketAddress().toString(),
           proxySock.getLocalSocketAddress().toString(),
-          false, "", null, datanode);
+          null, 0, 0, 0, "", null, datanode);
 
 
       // receive a block
       // receive a block
       blockReceiver.receiveBlock(null, null, null, null, 
       blockReceiver.receiveBlock(null, null, null, null, 

File diff suppressed because it is too large
+ 555 - 332
src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java


+ 103 - 7
src/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java

@@ -28,7 +28,10 @@ import java.io.OutputStream;
 
 
 
 
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 
 
@@ -93,6 +96,14 @@ public interface FSDatasetInterface extends FSDatasetMBean {
    */
    */
   public long getLength(Block b) throws IOException;
   public long getLength(Block b) throws IOException;
 
 
+  /**
+   * Get reference to the replica meta info in the replicasMap. 
+   * To be called from methods that are synchronized on {@link FSDataset}
+   * @param blockId
+   * @return replica from the replicas map
+   */
+  public Replica getReplica(long blockId);
+
   /**
   /**
    * @return the generation stamp stored with the block.
    * @return the generation stamp stored with the block.
    */
    */
@@ -144,6 +155,10 @@ public interface FSDatasetInterface extends FSDatasetMBean {
         checksumOut = cOut;
         checksumOut = cOut;
       }
       }
       
       
+      void close() throws IOException {
+        IOUtils.closeStream(dataOut);
+        IOUtils.closeStream(checksumOut);
+      }
     }
     }
 
 
   /**
   /**
@@ -167,15 +182,75 @@ public interface FSDatasetInterface extends FSDatasetMBean {
   }
   }
     
     
   /**
   /**
-   * Creates the block and returns output streams to write data and CRC
-   * @param b
-   * @param isRecovery True if this is part of erro recovery, otherwise false
-   * @return a BlockWriteStreams object to allow writing the block data
-   *  and CRC
+   * Creates a temporary replica and returns the meta information of the replica
+   * 
+   * @param b block
+   * @return the meta info of the replica which is being written to
+   * @throws IOException if an error occurs
+   */
+  public ReplicaInPipelineInterface createTemporary(Block b)
+  throws IOException;
+
+  /**
+   * Creates a RBW replica and returns the meta info of the replica
+   * 
+   * @param b block
+   * @return the meta info of the replica which is being written to
+   * @throws IOException if an error occurs
+   */
+  public ReplicaInPipelineInterface createRbw(Block b) throws IOException;
+
+  /**
+   * Recovers a RBW replica and returns the meta info of the replica
+   * 
+   * @param b block
+   * @param newGS the new generation stamp for the replica
+   * @param minBytesRcvd the minimum number of bytes that the replica could have
+   * @param maxBytesRcvd the maximum number of bytes that the replica could have
+   * @return the meta info of the replica which is being written to
+   * @throws IOException if an error occurs
+   */
+  public ReplicaInPipelineInterface recoverRbw(Block b, 
+      long newGS, long minBytesRcvd, long maxBytesRcvd)
+  throws IOException;
+
+  /**
+   * Append to a finalized replica and returns the meta info of the replica
+   * 
+   * @param b block
+   * @param newGS the new generation stamp for the replica
+   * @param expectedBlockLen the number of bytes the replica is expected to have
+   * @return the meata info of the replica which is being written to
    * @throws IOException
    * @throws IOException
    */
    */
-  public BlockWriteStreams writeToBlock(Block b, boolean isRecovery) throws IOException;
+  public ReplicaInPipelineInterface append(Block b, 
+      long newGS, long expectedBlockLen) throws IOException;
 
 
+  /**
+   * Recover a failed append to a finalized replica
+   * and returns the meta info of the replica
+   * 
+   * @param b block
+   * @param newGS the new generation stamp for the replica
+   * @param expectedBlockLen the number of bytes the replica is expected to have
+   * @return the meta info of the replica which is being written to
+   * @throws IOException
+   */
+  public ReplicaInPipelineInterface recoverAppend(Block b,
+      long newGS, long expectedBlockLen) throws IOException;
+  
+  /**
+   * Recover a failed pipeline close
+   * It bumps the replica's generation stamp and finalize it if RBW replica
+   * 
+   * @param b block
+   * @param newGS the new generation stamp for the replica
+   * @param expectedBlockLen the number of bytes the replica is expected to have
+   * @throws IOException
+   */
+  public void recoverClose(Block b,
+      long newGS, long expectedBlockLen) throws IOException;
+  
   /**
   /**
    * Update the block to the new generation stamp and length.  
    * Update the block to the new generation stamp and length.  
    */
    */
@@ -202,7 +277,7 @@ public interface FSDatasetInterface extends FSDatasetMBean {
    * Returns the block report - the full list of blocks stored
    * Returns the block report - the full list of blocks stored
    * @return - the block report - the full list of blocks stored
    * @return - the block report - the full list of blocks stored
    */
    */
-  public Block[] getBlockReport();
+  public BlockListAsLongs getBlockReport();
 
 
   /**
   /**
    * Is the block valid?
    * Is the block valid?
@@ -270,4 +345,25 @@ public interface FSDatasetInterface extends FSDatasetMBean {
    * @return true if more then minimum valid volumes left in the FSDataSet
    * @return true if more then minimum valid volumes left in the FSDataSet
    */
    */
   public boolean hasEnoughResource();
   public boolean hasEnoughResource();
+
+  /**
+   * Get visible length of the specified replica.
+   */
+  long getReplicaVisibleLength(final Block block) throws IOException;
+
+  /**
+   * Initialize a replica recovery.
+   * 
+   * @return actual state of the replica on this data-node or 
+   * null if data-node does not have the replica.
+   */
+  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
+  throws IOException;
+
+  /**
+   * Update replica's generation stamp and length and finalize it.
+   */
+  public FinalizedReplica updateReplicaUnderRecovery(Block oldBlock,
+                                          long recoveryId,
+                                          long newLength) throws IOException;
 }
 }

+ 95 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java

@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+
+/**
+ * This class describes a replica that has been finalized.
+ */
+class FinalizedReplica extends ReplicaInfo {
+  private boolean unlinked;      // copy-on-write done for block
+
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  FinalizedReplica(long blockId, long len, long genStamp,
+      FSVolume vol, File dir) {
+    super(blockId, len, genStamp, vol, dir);
+  }
+  
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  FinalizedReplica(Block block, FSVolume vol, File dir) {
+    super(block, vol, dir);
+  }
+  
+  @Override  // ReplicaInfo
+  public ReplicaState getState() {
+    return ReplicaState.FINALIZED;
+  }
+  
+  @Override // ReplicaInfo
+  boolean isUnlinked() {
+    return unlinked;
+  }
+
+  @Override  // ReplicaInfo
+  void setUnlinked() {
+    unlinked = true;
+  }
+  
+  @Override
+  public long getVisibleLength() {
+    return getNumBytes();       // all bytes are visible
+  }
+
+  @Override
+  public long getBytesOnDisk() {
+    return getNumBytes();
+  }
+
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+  
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+  
+  @Override
+  public String toString() {
+    return super.toString()
+        + "\n  unlinked=" + unlinked;
+  }
+}

+ 55 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/Replica.java

@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+
+/** 
+ * This represents block replicas which stored in DataNode.
+ */
+public interface Replica {
+  /** get block ID  */
+  public long getBlockId();
+
+  /** get generation stamp */
+  public long getGenerationStamp();
+
+  /**
+   * Get the replica state
+   * @return the replica state
+   */
+  public ReplicaState getState();
+
+  /**
+   * Get the number of bytes received
+   * @return the number of bytes that have been received
+   */
+  public long getNumBytes();
+  
+  /**
+   * Get the number of bytes that have written to disk
+   * @return the number of bytes that have written to disk
+   */
+  public long getBytesOnDisk();
+
+  /**
+   * Get the number of bytes that are visible to readers
+   * @return the number of bytes that are visible to readers
+   */
+  public long getVisibleLength();
+}

+ 3 - 3
src/java/org/apache/hadoop/hdfs/server/datanode/BlockAlreadyExistsException.java → src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaAlreadyExistsException.java

@@ -25,14 +25,14 @@ import java.io.IOException;
  * Exception indicating that the target block already exists 
  * Exception indicating that the target block already exists 
  * and is not set to be recovered/overwritten.  
  * and is not set to be recovered/overwritten.  
  */
  */
-class BlockAlreadyExistsException extends IOException {
+class ReplicaAlreadyExistsException extends IOException {
   private static final long serialVersionUID = 1L;
   private static final long serialVersionUID = 1L;
 
 
-  public BlockAlreadyExistsException() {
+  public ReplicaAlreadyExistsException() {
     super();
     super();
   }
   }
 
 
-  public BlockAlreadyExistsException(String msg) {
+  public ReplicaAlreadyExistsException(String msg) {
     super(msg);
     super(msg);
   }
   }
 }
 }

+ 88 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java

@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+
+/** This class represents replicas being written. 
+ * Those are the replicas that
+ * are created in a pipeline initiated by a dfs client.
+ */
+class ReplicaBeingWritten extends ReplicaInPipeline {
+  /**
+   * Constructor for a zero length replica
+   * @param blockId block id
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+    ReplicaBeingWritten(long blockId, long genStamp, 
+        FSVolume vol, File dir) {
+    super( blockId, genStamp, vol, dir);
+  }
+  
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param writer a thread that is writing to this replica
+   */
+  ReplicaBeingWritten(Block block, 
+      FSVolume vol, File dir, Thread writer) {
+    super( block, vol, dir, writer);
+  }
+
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param writer a thread that is writing to this replica
+   */
+  ReplicaBeingWritten(long blockId, long len, long genStamp,
+      FSVolume vol, File dir, Thread writer ) {
+    super( blockId, len, genStamp, vol, dir, writer);
+  }
+  
+  @Override
+  public long getVisibleLength() {
+    return getBytesAcked();       // all acked bytes are visible
+  }
+
+  @Override   //ReplicaInfo
+  public ReplicaState getState() {
+    return ReplicaState.RBW;
+  }
+  
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+  
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+}

+ 182 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java

@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.BlockWriteStreams;
+import org.apache.hadoop.io.IOUtils;
+
+/** 
+ * This class defines a replica in a pipeline, which
+ * includes a persistent replica being written to by a dfs client or
+ * a temporary replica being replicated by a source datanode or
+ * being copied for the balancing purpose.
+ * 
+ * The base class implements a temporary replica
+ */
+class ReplicaInPipeline extends ReplicaInfo
+                        implements ReplicaInPipelineInterface {
+  private long bytesAcked;
+  private long bytesOnDisk;
+  private Thread writer;
+  
+  /**
+   * Constructor for a zero length replica
+   * @param blockId block id
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param state replica state
+   */
+    ReplicaInPipeline(long blockId, long genStamp, 
+        FSVolume vol, File dir) {
+    this( blockId, 0L, genStamp, vol, dir, Thread.currentThread());
+  }
+
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param writer a thread that is writing to this replica
+   */
+  ReplicaInPipeline(Block block, 
+      FSVolume vol, File dir, Thread writer) {
+    this( block.getBlockId(), block.getNumBytes(), block.getGenerationStamp(),
+        vol, dir, writer);
+  }
+
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   * @param writer a thread that is writing to this replica
+   */
+  ReplicaInPipeline(long blockId, long len, long genStamp,
+      FSVolume vol, File dir, Thread writer ) {
+    super( blockId, len, genStamp, vol, dir);
+    this.bytesAcked = len;
+    this.bytesOnDisk = len;
+    this.writer = writer;
+  }
+
+  @Override
+  public long getVisibleLength() {
+    return -1;
+  }
+  
+  @Override  //ReplicaInfo
+  public ReplicaState getState() {
+    return ReplicaState.TEMPORARY;
+  }
+  
+  @Override // ReplicaInPipelineInterface
+  public long getBytesAcked() {
+    return bytesAcked;
+  }
+  
+  @Override // ReplicaInPipelineInterface
+  public void setBytesAcked(long bytesAcked) {
+    this.bytesAcked = bytesAcked;
+  }
+  
+  @Override // ReplicaInPipelineInterface
+  public long getBytesOnDisk() {
+    return bytesOnDisk;
+  }
+  
+  @Override //ReplicaInPipelineInterface
+  public void setBytesOnDisk(long bytesOnDisk) {
+    this.bytesOnDisk = bytesOnDisk;
+  }
+  
+  /**
+   * Set the thread that is writing to this replica
+   * @param writer a thread writing to this replica
+   */
+  public void setWriter(Thread writer) {
+    this.writer = writer;
+  }
+  
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+  
+  /**
+   * Interrupt the writing thread and wait until it dies
+   * @throws IOException the waiting is interrupted
+   */
+  void stopWriter() throws IOException {
+    if (writer != null && writer != Thread.currentThread() && writer.isAlive()) {
+      writer.interrupt();
+      try {
+        writer.join();
+      } catch (InterruptedException e) {
+        throw new IOException("Waiting for writer thread is interrupted.");
+      }
+    }
+  }
+  
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+  
+  @Override // ReplicaInPipelineInterface
+  public BlockWriteStreams createStreams() throws IOException {
+    File blockFile = getBlockFile();
+    File metaFile = getMetaFile();
+    if (DataNode.LOG.isDebugEnabled()) {
+      DataNode.LOG.debug("writeTo blockfile is " + blockFile +
+                         " of size " + blockFile.length());
+      DataNode.LOG.debug("writeTo metafile is " + metaFile +
+                         " of size " + metaFile.length());
+    }
+    FileOutputStream blockOut = null;
+    FileOutputStream crcOut = null;
+    try {
+      blockOut = new FileOutputStream(
+          new RandomAccessFile( blockFile, "rw" ).getFD() );
+      crcOut = new FileOutputStream(
+          new RandomAccessFile( metaFile, "rw" ).getFD() );
+      return new BlockWriteStreams(blockOut, crcOut);
+    } catch (IOException e) {
+      IOUtils.closeStream(blockOut);
+      IOUtils.closeStream(crcOut);
+      throw e;
+    }
+  }
+  
+  @Override
+  public String toString() {
+    return super.toString()
+        + "\n  bytesAcked=" + bytesAcked
+        + "\n  bytesOnDisk=" + bytesOnDisk;
+  }
+}

+ 60 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java

@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.BlockWriteStreams;
+
+/** 
+ * This defines the interface of a replica in Pipeline that's being written to
+ */
+interface ReplicaInPipelineInterface extends Replica {
+  /**
+   * Set the number of bytes received
+   * @param bytesReceived number of bytes received
+   */
+  void setNumBytes(long bytesReceived);
+  
+  /**
+   * Get the number of bytes acked
+   * @return the number of bytes acked
+   */
+  long getBytesAcked();
+  
+  /**
+   * Set the number bytes that have acked
+   * @param bytesAcked
+   */
+  void setBytesAcked(long bytesAcked);
+  
+  /**
+   * Set the number of bytes on disk
+   * @param bytesOnDisk number of bytes on disk
+   */
+  void setBytesOnDisk(long bytesOnDisk);
+  
+  /**
+   * Create output streams for writing to this replica, 
+   * one for block file and one for CRC file
+   * 
+   * @return output streams for writing
+   * @throws IOException if any error occurs
+   */
+  public BlockWriteStreams createStreams() throws IOException;
+}

+ 140 - 45
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -22,69 +22,137 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil.HardLink;
 import org.apache.hadoop.fs.FileUtil.HardLink;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 
 
 /**
 /**
- * This class is used by the datanode to maintain the map from a block 
- * to its metadata.
+ * This class is used by datanodes to maintain meta data of its replicas.
+ * It provides a general interface for meta information of a replica.
  */
  */
-class ReplicaInfo {
-
-  private FSVolume volume;       // volume where the block belongs
-  private File     file;         // block file
-  private boolean detached;      // copy-on-write done for block
+abstract public class ReplicaInfo extends Block implements Replica {
+  private FSVolume volume;      // volume where the replica belongs
+  private File     dir;         // directory where block & meta files belong
 
 
-  ReplicaInfo(FSVolume vol, File file) {
-    this.volume = vol;
-    this.file = file;
-    detached = false;
+  /**
+   * Constructor for a zero length replica
+   * @param blockId block id
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaInfo(long blockId, long genStamp, FSVolume vol, File dir) {
+    this( blockId, 0L, genStamp, vol, dir);
   }
   }
-
-  ReplicaInfo(FSVolume vol) {
+  
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaInfo(Block block, FSVolume vol, File dir) {
+    this(block.getBlockId(), block.getNumBytes(), 
+        block.getGenerationStamp(), vol, dir);
+  }
+  
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaInfo(long blockId, long len, long genStamp,
+      FSVolume vol, File dir) {
+    super(blockId, len, genStamp);
     this.volume = vol;
     this.volume = vol;
-    this.file = null;
-    detached = false;
+    this.dir = dir;
   }
   }
 
 
+  /**
+   * Get this replica's meta file name
+   * @return this replica's meta file name
+   */
+  private String getMetaFileName() {
+    return getBlockName() + "_" + getGenerationStamp() + METADATA_EXTENSION; 
+  }
+  
+  /**
+   * Get the full path of this replica's data file
+   * @return the full path of this replica's data file
+   */
+  File getBlockFile() {
+    return new File(getDir(), getBlockName());
+  }
+  
+  /**
+   * Get the full path of this replica's meta file
+   * @return the full path of this replica's meta file
+   */
+  File getMetaFile() {
+    return new File(getDir(), getMetaFileName());
+  }
+  
+  /**
+   * Get the volume where this replica is located on disk
+   * @return the volume where this replica is located on disk
+   */
   FSVolume getVolume() {
   FSVolume getVolume() {
     return volume;
     return volume;
   }
   }
-
-  File getFile() {
-    return file;
+  
+  /**
+   * Set the volume where this replica is located on disk
+   */
+  void setVolume(FSVolume vol) {
+    this.volume = vol;
   }
   }
-
-  void setFile(File f) {
-    file = f;
+  
+  /**
+   * Return the parent directory path where this replica is located
+   * @return the parent directory path where this replica is located
+   */
+  File getDir() {
+    return dir;
   }
   }
 
 
   /**
   /**
-   * Is this block already detached?
+   * Set the parent directory where this replica is located
+   * @param dir the parent directory where the replica is located
    */
    */
-  boolean isDetached() {
-    return detached;
+  void setDir(File dir) {
+    this.dir = dir;
   }
   }
 
 
   /**
   /**
-   *  Block has been successfully detached
+   * check if this replica has already been unlinked.
+   * @return true if the replica has already been unlinked 
+   *         or no need to be detached; false otherwise
    */
    */
-  void setDetached() {
-    detached = true;
+  boolean isUnlinked() {
+    return true;                // no need to be unlinked
   }
   }
 
 
   /**
   /**
+   * set that this replica is unlinked
+   */
+  void setUnlinked() {
+    // no need to be unlinked
+  }
+  
+   /**
    * Copy specified file into a temporary file. Then rename the
    * Copy specified file into a temporary file. Then rename the
    * temporary file to the original name. This will cause any
    * temporary file to the original name. This will cause any
    * hardlinks to the original file to be removed. The temporary
    * hardlinks to the original file to be removed. The temporary
-   * files are created in the detachDir. The temporary files will
+   * files are created in the same directory. The temporary files will
    * be recovered (especially on Windows) on datanode restart.
    * be recovered (especially on Windows) on datanode restart.
    */
    */
-  private void detachFile(File file, Block b) throws IOException {
-    File tmpFile = volume.createDetachFile(b, file.getName());
+  private void unlinkFile(File file, Block b) throws IOException {
+    File tmpFile = FSDataset.createTmpFile(b, FSDataset.getUnlinkTmpFile(file));
     try {
     try {
       FileInputStream in = new FileInputStream(file);
       FileInputStream in = new FileInputStream(file);
       try {
       try {
@@ -114,33 +182,60 @@ class ReplicaInfo {
   }
   }
 
 
   /**
   /**
-   * Returns true if this block was copied, otherwise returns false.
+   * Remove a hard link by copying the block to a temporary place and 
+   * then moving it back
+   * @param numLinks number of hard links
+   * @return true if copy is successful; 
+   *         false if it is already detached or no need to be detached
+   * @throws IOException if there is any copy error
    */
    */
-  boolean detachBlock(Block block, int numLinks) throws IOException {
-    if (isDetached()) {
+  boolean unlinkBlock(int numLinks) throws IOException {
+    if (isUnlinked()) {
       return false;
       return false;
     }
     }
-    if (file == null || volume == null) {
-      throw new IOException("detachBlock:Block not found. " + block);
+    File file = getBlockFile();
+    if (file == null || getVolume() == null) {
+      throw new IOException("detachBlock:Block not found. " + this);
     }
     }
-    File meta = FSDataset.getMetaFile(file, block);
+    File meta = getMetaFile();
     if (meta == null) {
     if (meta == null) {
-      throw new IOException("Meta file not found for block " + block);
+      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 " + block);
-      detachFile(file, block);
+      DataNode.LOG.info("CopyOnWrite for block " + this);
+      unlinkFile(file, this);
     }
     }
     if (HardLink.getLinkCount(meta) > numLinks) {
     if (HardLink.getLinkCount(meta) > numLinks) {
-      detachFile(meta, block);
+      unlinkFile(meta, this);
     }
     }
-    setDetached();
+    setUnlinked();
     return true;
     return true;
   }
   }
+
+  /**
+   * Set this replica's generation stamp to be a newer one
+   * @param newGS new generation stamp
+   * @throws IOException is the new generation stamp is not greater than the current one
+   */
+  void setNewerGenerationStamp(long newGS) throws IOException {
+    long curGS = getGenerationStamp();
+    if (newGS <= curGS) {
+      throw new IOException("New generation stamp (" + newGS 
+          + ") must be greater than current one (" + curGS + ")");
+    }
+    setGenerationStamp(newGS);
+  }
   
   
+  @Override  //Object
   public String toString() {
   public String toString() {
-    return getClass().getSimpleName() + "(volume=" + volume
-        + ", file=" + file + ", detached=" + detached + ")";
+    return getClass().getSimpleName()
+        + ", " + super.toString()
+        + ", " + getState()
+        + "\n  getNumBytes()     = " + getNumBytes()
+        + "\n  getBytesOnDisk()  = " + getBytesOnDisk()
+        + "\n  getVisibleLength()= " + getVisibleLength()
+        + "\n  getVolume()       = " + getVolume()
+        + "\n  getBlockFile()    = " + getBlockFile();
   }
   }
 }
 }

+ 53 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Exception indicating that DataNode does not have a replica
+ * that matches the target block.  
+ */
+class ReplicaNotFoundException extends IOException {
+  private static final long serialVersionUID = 1L;
+  final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
+  final static String UNFINALIZED_REPLICA = 
+    "Cannot append to an unfinalized replica ";
+  final static String UNFINALIZED_AND_NONRBW_REPLICA = 
+    "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
+  final static String NON_EXISTENT_REPLICA =
+    "Cannot append to a non-existent replica ";
+  final static String UNEXPECTED_GS_REPLICA =
+    "Cannot append to a replica with unexpeted generation stamp ";
+
+  public ReplicaNotFoundException() {
+    super();
+  }
+
+  ReplicaNotFoundException(Block b) {
+    super("Replica not found for " + b);
+  }
+  
+  public ReplicaNotFoundException(String msg) {
+    super(msg);
+  }
+}

+ 163 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java

@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+
+/**
+ * This class represents replicas that are under block recovery
+ * It has a recovery id that is equal to the generation stamp 
+ * that the replica will be bumped to after recovery
+ * The recovery id is used to handle multiple concurrent block recoveries.
+ * A recovery with higher recovery id preempts recoveries with a lower id.
+ *
+ */
+class ReplicaUnderRecovery extends ReplicaInfo {
+  private ReplicaInfo original; // the original replica that needs to be recovered
+  private long recoveryId; // recovery id; it is also the generation stamp 
+                           // that the replica will be bumped to after recovery
+
+  ReplicaUnderRecovery(ReplicaInfo replica, long recoveryId) {
+    super(replica.getBlockId(), replica.getNumBytes(), replica.getGenerationStamp(),
+        replica.getVolume(), replica.getDir());
+    if ( replica.getState() != ReplicaState.FINALIZED &&
+         replica.getState() != ReplicaState.RBW &&
+         replica.getState() != ReplicaState.RWR ) {
+      throw new IllegalArgumentException("Cannot recover replica: " + replica);
+    }
+    this.original = replica;
+    this.recoveryId = recoveryId;
+  }
+
+  /** 
+   * Get the recovery id
+   * @return the generation stamp that the replica will be bumped to 
+   */
+  long getRecoveryID() {
+    return recoveryId;
+  }
+
+  /** 
+   * Set the recovery id
+   * @param recoveryId the new recoveryId
+   */
+  void setRecoveryID(long recoveryId) {
+    if (recoveryId > this.recoveryId) {
+      this.recoveryId = recoveryId;
+    } else {
+      throw new IllegalArgumentException("The new rcovery id: " + recoveryId
+          + " must be greater than the current one: " + this.recoveryId);
+    }
+  }
+
+  /**
+   * Get the original replica that's under recovery
+   * @return the original replica under recovery
+   */
+  ReplicaInfo getOriginalReplica() {
+    return original;
+  }
+  
+  /**
+   * Get the original replica's state
+   * @return the original replica's state
+   */
+  ReplicaState getOrignalReplicaState() {
+    return original.getState();
+  }
+
+  @Override //ReplicaInfo
+  boolean isUnlinked() {
+    return original.isUnlinked();
+  }
+
+  @Override //ReplicaInfo
+  void setUnlinked() {
+    original.setUnlinked();
+  }
+  
+  @Override //ReplicaInfo
+  public ReplicaState getState() {
+    return ReplicaState.RUR;
+  }
+  
+  @Override
+  public long getVisibleLength() {
+    return original.getVisibleLength();
+  }
+
+  @Override
+  public long getBytesOnDisk() {
+    return original.getBytesOnDisk();
+  }
+
+  @Override  //org.apache.hadoop.hdfs.protocol.Block
+  public void setBlockId(long blockId) {
+    super.setBlockId(blockId);
+    original.setBlockId(blockId);
+  }
+
+  @Override //org.apache.hadoop.hdfs.protocol.Block
+  public void setGenerationStamp(long gs) {
+    super.setGenerationStamp(gs);
+    original.setGenerationStamp(gs);
+  }
+  
+  @Override //org.apache.hadoop.hdfs.protocol.Block
+  public void setNumBytes(long numBytes) {
+    super.setNumBytes(numBytes);
+    original.setNumBytes(numBytes);
+  }
+  
+  @Override //ReplicaInfo
+  void setDir(File dir) {
+    super.setDir(dir);
+    original.setDir(dir);
+  }
+  
+  @Override //ReplicaInfo
+  void setVolume(FSVolume vol) {
+    super.setVolume(vol);
+    original.setVolume(vol);
+  }
+  
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+  
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return super.toString()
+        + "\n  recoveryId=" + recoveryId
+        + "\n  original=" + original;
+  }
+
+  ReplicaRecoveryInfo createInfo() {
+    return new ReplicaRecoveryInfo(this, getOrignalReplicaState()); 
+  }
+}

+ 101 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java

@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+
+/**
+ * This class represents a replica that is waiting to be recovered.
+ * After a datanode restart, any replica in "rbw" directory is loaded
+ * as a replica waiting to be recovered.
+ * A replica waiting to be recovered does not provision read nor
+ * participates in any pipeline recovery. It will become outdated if its
+ * client continues to write or be recovered as a result of
+ * lease recovery.
+ */
+class ReplicaWaitingToBeRecovered extends ReplicaInfo {
+  private boolean unlinked;      // copy-on-write done for block
+
+  /**
+   * Constructor
+   * @param blockId block id
+   * @param len replica length
+   * @param genStamp replica generation stamp
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaWaitingToBeRecovered(long blockId, long len, long genStamp,
+      FSVolume vol, File dir) {
+    super(blockId, len, genStamp, vol, dir);
+  }
+  
+  /**
+   * Constructor
+   * @param block a block
+   * @param vol volume where replica is located
+   * @param dir directory path where block and meta files are located
+   */
+  ReplicaWaitingToBeRecovered(Block block, FSVolume vol, File dir) {
+    super(block, vol, dir);
+  }
+  
+  @Override //ReplicaInfo
+  public ReplicaState getState() {
+    return ReplicaState.RWR;
+  }
+  
+  @Override //ReplicaInfo
+  boolean isUnlinked() {
+    return unlinked;
+  }
+
+  @Override //ReplicaInfo
+  void setUnlinked() {
+    unlinked = true;
+  }
+  
+  @Override //ReplicaInfo
+  public long getVisibleLength() {
+    return -1;  //no bytes are visible
+  }
+  
+  @Override
+  public long getBytesOnDisk() {
+    return getNumBytes();
+  }
+
+  @Override  // Object
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+  
+  @Override  // Object
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return super.toString()
+        + "\n  unlinked=" + unlinked;
+  }
+}

+ 115 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicasMap.java

@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.util.Collection;
+import java.util.HashMap;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+class ReplicasMap {
+  // HashMap: maps a block id to the replica's meta info
+  private HashMap<Long, ReplicaInfo> map = new HashMap<Long, ReplicaInfo>();
+  /**
+   * Get the meta information of the replica that matches both block id 
+   * and generation stamp
+   * @param block block with its id as the key
+   * @return the replica's meta information
+   * @throws IllegalArgumentException if the input block is null
+   */
+  ReplicaInfo get(Block block) {
+    if (block == null) {
+      throw new IllegalArgumentException("Do not expect null block");
+    }
+    ReplicaInfo replicaInfo = get(block.getBlockId());
+    if (replicaInfo != null && 
+        block.getGenerationStamp() == replicaInfo.getGenerationStamp()) {
+      return replicaInfo;
+    }
+    return null;
+  }
+  
+  /**
+   * Get the meta information of the replica that matches the block id
+   * @param blockId a block's id
+   * @return the replica's meta information
+   */
+  ReplicaInfo get(long blockId) {
+    return map.get(blockId);
+  }
+  
+  /**
+   * Add a replica's meta information into the map 
+   * 
+   * @param replicaInfo a replica's meta information
+   * @return previous meta information of the replica
+   * @throws IllegalArgumentException if the input parameter is null
+   */
+  ReplicaInfo add(ReplicaInfo replicaInfo) {
+    if (replicaInfo == null) {
+      throw new IllegalArgumentException("Do not expect null block");
+    }
+    return  map.put(replicaInfo.getBlockId(), replicaInfo);
+  }
+  
+  /**
+   * Remove the replica's meta information from the map that matches
+   * the input block's id and generation stamp
+   * @param block block with its id as the key
+   * @return the removed replica's meta information
+   * @throws IllegalArgumentException if the input block is null
+   */
+  ReplicaInfo remove(Block block) {
+    if (block == null) {
+      throw new IllegalArgumentException("Do not expect null block");
+    }
+    Long key = Long.valueOf(block.getBlockId());
+    ReplicaInfo replicaInfo = map.get(key);
+    if (replicaInfo != null &&
+        replicaInfo.getGenerationStamp() == replicaInfo.getGenerationStamp()) {
+      return remove(key);
+    } 
+    
+    return null;
+  }
+  
+  /**
+   * Remove the replica's meta information from the map if present
+   * @param the block id of the replica to be removed
+   * @return the removed replica's meta information
+   */
+  ReplicaInfo remove(long blockId) {
+    return map.remove(blockId);
+  }
+ 
+  /**
+   * Get the size of the map
+   * @return the number of replicas in the map
+   */
+  int size() {
+    return map.size();
+  }
+  
+  /**
+   * Get a collection of the replicas
+   * @return a collection of the replicas
+   */
+  Collection<ReplicaInfo> replicas() {
+    return map.values();
+  }
+}

+ 51 - 3
src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 
 
 /**
 /**
  * Internal class for block metadata.
  * Internal class for block metadata.
@@ -35,12 +36,22 @@ class BlockInfo extends Block {
    */
    */
   private Object[] triplets;
   private Object[] triplets;
 
 
-  BlockInfo(Block blk, int replication) {
+  protected BlockInfo(Block blk, int replication) {
     super(blk);
     super(blk);
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
     this.inode = null;
     this.inode = null;
   }
   }
 
 
+  /**
+   * Copy construction.
+   * This is used to convert BlockInfoUnderConstruction
+   * @param from BlockInfo to copy from.
+   */
+  protected BlockInfo(BlockInfo from) {
+    this(from, from.inode.getReplication());
+    this.inode = from.inode;
+  }
+
   INodeFile getINode() {
   INodeFile getINode() {
     return inode;
     return inode;
   }
   }
@@ -64,7 +75,7 @@ class BlockInfo extends Block {
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
     BlockInfo info = (BlockInfo)triplets[index*3+1];
     assert info == null || 
     assert info == null || 
-        BlockInfo.class.getName().equals(info.getClass().getName()) : 
+        info.getClass().getName().startsWith(BlockInfo.class.getName()) : 
               "BlockInfo is expected at " + index*3;
               "BlockInfo is expected at " + index*3;
     return info;
     return info;
   }
   }
@@ -74,7 +85,7 @@ class BlockInfo extends Block {
     assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
     assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+2];
     BlockInfo info = (BlockInfo)triplets[index*3+2];
     assert info == null || 
     assert info == null || 
-        BlockInfo.class.getName().equals(info.getClass().getName()) : 
+        info.getClass().getName().startsWith(BlockInfo.class.getName()) : 
               "BlockInfo is expected at " + index*3;
               "BlockInfo is expected at " + index*3;
     return info;
     return info;
   }
   }
@@ -262,6 +273,43 @@ class BlockInfo extends Block {
     return true;
     return true;
   }
   }
 
 
+  /**
+   * BlockInfo represents a block that is not being constructed.
+   * In order to start modifying the block, the BlockInfo should be converted
+   * to {@link BlockInfoUnderConstruction}.
+   * @return {@link BlockUCState#COMPLETE}
+   */
+  BlockUCState getBlockUCState() {
+    return BlockUCState.COMPLETE;
+  }
+
+  /**
+   * Is this block complete?
+   * 
+   * @return true if the state of the block is {@link BlockUCState#COMPLETE}
+   */
+  boolean isComplete() {
+    return getBlockUCState().equals(BlockUCState.COMPLETE);
+  }
+
+  /**
+   * Convert a complete block to an under construction block.
+   * 
+   * @return BlockInfoUnderConstruction -  an under construction block.
+   */
+  BlockInfoUnderConstruction convertToBlockUnderConstruction(
+      BlockUCState s, DatanodeDescriptor[] targets) {
+    if(isComplete()) {
+      return new BlockInfoUnderConstruction(
+          this, getINode().getReplication(), s, targets);
+    }
+    // the block is already under construction
+    BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;
+    ucBlock.setBlockUCState(s);
+    ucBlock.setExpectedLocations(targets);
+    return ucBlock;
+  }
+
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     // Super implementation is sufficient
     // Super implementation is sufficient

+ 266 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java

@@ -0,0 +1,266 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+
+/**
+ * Represents a block that is currently being constructed.<br>
+ * This is usually the last block of a file opened for write or append.
+ */
+class BlockInfoUnderConstruction extends BlockInfo {
+  /** Block state. See {@link BlockUCState} */
+  private BlockUCState blockUCState;
+
+  /**
+   * Block replicas as assigned when the block was allocated.
+   * This defines the pipeline order.
+   */
+  private List<ReplicaUnderConstruction> replicas;
+
+  /** A data-node responsible for block recovery. */
+  private int primaryNodeIndex = -1;
+
+  /**
+   * The new generation stamp, which this block will have
+   * after the recovery succeeds. Also used as a recovery id to identify
+   * the right recovery if any of the abandoned recoveries re-appear.
+   */
+  private long blockRecoveryId = 0;
+
+  /**
+   * ReplicaUnderConstruction contains information about replicas while
+   * they are under construction.
+   * The GS, the length and the state of the replica is as reported by 
+   * the data-node.
+   * It is not guaranteed, but expected, that data-nodes actually have
+   * corresponding replicas.
+   */
+  static class ReplicaUnderConstruction extends Block {
+    private DatanodeDescriptor expectedLocation;
+    private ReplicaState state;
+
+    ReplicaUnderConstruction(Block block,
+                             DatanodeDescriptor target,
+                             ReplicaState state) {
+      super(block);
+      this.expectedLocation = target;
+      this.state = state;
+    }
+
+    /**
+     * Expected block replica location as assigned when the block was allocated.
+     * This defines the pipeline order.
+     * It is not guaranteed, but expected, that the data-node actually has
+     * the replica.
+     */
+    DatanodeDescriptor getExpectedLocation() {
+      return expectedLocation;
+    }
+
+    /**
+     * Get replica state as reported by the data-node.
+     */
+    ReplicaState getState() {
+      return state;
+    }
+
+    /**
+     * Set replica state.
+     */
+    void setState(ReplicaState s) {
+      state = s;
+    }
+
+    /**
+     * Is data-node the replica belongs to alive.
+     */
+    boolean isAlive() {
+      return expectedLocation.isAlive;
+    }
+
+    @Override // Block
+    public int hashCode() {
+      return super.hashCode();
+    }
+
+    @Override // Block
+    public boolean equals(Object obj) {
+      // Sufficient to rely on super's implementation
+      return (this == obj) || super.equals(obj);
+    }
+  }
+
+  /**
+   * Create block and set its state to
+   * {@link BlockUCState#UNDER_CONSTRUCTION}.
+   */
+  BlockInfoUnderConstruction(Block blk, int replication) {
+    this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
+  }
+
+  BlockInfoUnderConstruction(Block blk, int replication,
+                             BlockUCState state,
+                             DatanodeDescriptor[] targets) {
+    super(blk, replication);
+    assert getBlockUCState() != BlockUCState.COMPLETE :
+      "BlockInfoUnderConstruction cannot be in COMPLETE state";
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /**
+   * Convert an under construction block to a complete block.
+   * 
+   * @return BlockInfo - a complete block.
+   * @throws IOException if the state of the block 
+   * (the generation stamp and the length) has not been committed by 
+   * the client or it does not have at least a minimal number of replicas 
+   * reported from data-nodes. 
+   */
+  BlockInfo convertToCompleteBlock() throws IOException {
+    assert getBlockUCState() != BlockUCState.COMPLETE :
+      "Trying to convert a COMPLETE block";
+    if(getBlockUCState() != BlockUCState.COMMITTED)
+      throw new IOException(
+          "Cannot complete block: block has not been COMMITTED by the client");
+    return new BlockInfo(this);
+  }
+
+  void setExpectedLocations(DatanodeDescriptor[] targets) {
+    int numLocations = targets == null ? 0 : targets.length;
+    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
+    for(int i = 0; i < numLocations; i++)
+      replicas.add(
+        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
+  }
+
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  DatanodeDescriptor[] getExpectedLocations() {
+    int numLocations = replicas == null ? 0 : replicas.size();
+    DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
+    for(int i = 0; i < numLocations; i++)
+      locations[i] = replicas.get(i).getExpectedLocation();
+    return locations;
+  }
+
+  int getNumExpectedLocations() {
+    return replicas == null ? 0 : replicas.size();
+  }
+
+  /**
+   * Return the state of the block under construction.
+   * @see BlockUCState
+   */
+  @Override // BlockInfo
+  BlockUCState getBlockUCState() {
+    return blockUCState;
+  }
+
+  void setBlockUCState(BlockUCState s) {
+    blockUCState = s;
+  }
+
+  long getBlockRecoveryId() {
+    return blockRecoveryId;
+  }
+
+  /**
+   * Commit block's length and generation stamp as reported by the client.
+   * Set block state to {@link BlockUCState#COMMITTED}.
+   * @param block - contains client reported block length and generation 
+   * @throws IOException if block ids are inconsistent.
+   */
+  void commitBlock(Block block) throws IOException {
+    if(getBlockId() != block.getBlockId())
+      throw new IOException("Trying to commit inconsistent block: id = "
+          + block.getBlockId() + ", expected id = " + getBlockId());
+    blockUCState = BlockUCState.COMMITTED;
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+  }
+
+  /**
+   * Initialize lease recovery for this block.
+   * Find the first alive data-node starting from the previous primary and
+   * make it primary.
+   */
+  void initializeBlockRecovery(long recoveryId) {
+    setBlockUCState(BlockUCState.UNDER_RECOVERY);
+    blockRecoveryId = recoveryId;
+    if (replicas.size() == 0) {
+      NameNode.stateChangeLog.warn("BLOCK*"
+        + " INodeFileUnderConstruction.initLeaseRecovery:"
+        + " No blocks found, lease removed.");
+    }
+
+    int previous = primaryNodeIndex;
+    for(int i = 1; i <= replicas.size(); i++) {
+      int j = (previous + i)%replicas.size();
+      if (replicas.get(j).isAlive()) {
+        primaryNodeIndex = j;
+        DatanodeDescriptor primary = replicas.get(j).getExpectedLocation(); 
+        primary.addBlockToBeRecovered(this);
+        NameNode.stateChangeLog.info("BLOCK* " + this
+          + " recovery started, primary=" + primary);
+        return;
+      }
+    }
+  }
+
+  void addReplicaIfNotPresent(DatanodeDescriptor dn,
+                     Block block,
+                     ReplicaState rState) {
+    for(ReplicaUnderConstruction r : replicas)
+      if(r.getExpectedLocation() == dn)
+        return;
+    replicas.add(new ReplicaUnderConstruction(block, dn, rState));
+  }
+
+  @Override // BlockInfo
+  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override // BlockInfo
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
+    b.append("{")
+     .append("\n  blockUCState=").append(blockUCState)
+     .append("\n  replicas=").append(replicas)
+     .append("\n  primaryNodeIndex=").append(primaryNodeIndex)
+     .append("}");
+    return b.toString();
+  }
+}

+ 170 - 47
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -22,7 +22,6 @@ import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
@@ -37,9 +36,11 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
 import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
-import org.apache.hadoop.security.AccessTokenHandler;
 
 
 /**
 /**
  * Keeps information related to the blocks stored in the Hadoop cluster.
  * Keeps information related to the blocks stored in the Hadoop cluster.
@@ -236,6 +237,80 @@ public class BlockManager {
     return (countNodes(block).liveReplicas() >= minReplication);
     return (countNodes(block).liveReplicas() >= minReplication);
   }
   }
 
 
+  /**
+   * Commit the last block of the file.
+   * 
+   * @param fileINode file inode
+   * @param commitBlock - contains client reported block length and generation
+   * @throws IOException if the block does not have at least a minimal number
+   * of replicas reported from data-nodes.
+   */
+  void commitLastBlock(INodeFileUnderConstruction fileINode, 
+                       Block commitBlock) throws IOException {
+    if(commitBlock == null)
+      return; // not committing, this is a block allocation retry
+    BlockInfo lastBlock = fileINode.getLastBlock();
+    if(lastBlock == null)
+      return; // no blocks in file yet
+    if(lastBlock.isComplete())
+      return; // already completed (e.g. by syncBlock)
+    assert lastBlock.getNumBytes() <= commitBlock.getNumBytes() :
+      "commitBlock length is less than the stored one "
+      + commitBlock.getNumBytes() + " vs. " + lastBlock.getNumBytes();
+    ((BlockInfoUnderConstruction)lastBlock).commitBlock(commitBlock);
+  }
+
+  /**
+   * Convert a specified block of the file to a complete block.
+   * @param fileINode file
+   * @param blkIndex  block index in the file
+   * @throws IOException if the block does not have at least a minimal number
+   * of replicas reported from data-nodes.
+   */
+  BlockInfo completeBlock(INodeFile fileINode, int blkIndex)
+  throws IOException {
+    if(blkIndex < 0)
+      return null;
+    BlockInfo curBlock = fileINode.getBlocks()[blkIndex];
+    if(curBlock.isComplete())
+      return curBlock;
+    BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
+    if(ucBlock.numNodes() < minReplication)
+      throw new IOException("Cannot complete block: " +
+          "block does not satisfy minimal replication requirement.");
+    BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
+    // replace penultimate block in file
+    fileINode.setBlock(blkIndex, completeBlock);
+    // replace block in the blocksMap
+    return blocksMap.replaceBlock(completeBlock);
+  }
+
+  BlockInfo completeBlock(INodeFile fileINode, BlockInfo block)
+  throws IOException {
+    BlockInfo[] fileBlocks = fileINode.getBlocks();
+    for(int idx = 0; idx < fileBlocks.length; idx++)
+      if(fileBlocks[idx] == block) {
+        return completeBlock(fileINode, idx);
+      }
+    return block;
+  }
+
+  /**
+   * Convert the last block of the file to an under construction block.
+   * @param fileINode file
+   * @param targets data-nodes that will form the pipeline for this block
+   */
+  void convertLastBlockToUnderConstruction(
+      INodeFileUnderConstruction fileINode,
+      DatanodeDescriptor[] targets) throws IOException {
+    BlockInfo oldBlock = fileINode.getLastBlock();
+    if(oldBlock == null)
+      return;
+    BlockInfoUnderConstruction ucBlock =
+      fileINode.setLastBlock(oldBlock, targets);
+    blocksMap.replaceBlock(ucBlock);
+  }
+
   /**
   /**
    * Get all valid locations of the block
    * Get all valid locations of the block
    */
    */
@@ -254,7 +329,7 @@ public class BlockManager {
     return machineSet;
     return machineSet;
   }
   }
 
 
-  List<LocatedBlock> getBlockLocations(Block[] blocks, long offset,
+  List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
       long length, int nrBlocksToReturn) throws IOException {
       long length, int nrBlocksToReturn) throws IOException {
     int curBlk = 0;
     int curBlk = 0;
     long curPos = 0, blkSize = 0;
     long curPos = 0, blkSize = 0;
@@ -269,43 +344,12 @@ public class BlockManager {
     }
     }
 
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return null;
+      return Collections.<LocatedBlock>emptyList();
 
 
     long endOff = offset + length;
     long endOff = offset + length;
     List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
     List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
     do {
     do {
-      // get block locations
-      int numNodes = blocksMap.numNodes(blocks[curBlk]);
-      int numCorruptNodes = countNodes(blocks[curBlk]).corruptReplicas();
-      int numCorruptReplicas = corruptReplicas
-          .numCorruptReplicas(blocks[curBlk]);
-      if (numCorruptNodes != numCorruptReplicas) {
-        FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for "
-            + blocks[curBlk] + "blockMap has " + numCorruptNodes
-            + " but corrupt replicas map has " + numCorruptReplicas);
-      }
-      boolean blockCorrupt = (numCorruptNodes == numNodes);
-      int numMachineSet = blockCorrupt ? numNodes :
-                          (numNodes - numCorruptNodes);
-      DatanodeDescriptor[] machineSet = new DatanodeDescriptor[numMachineSet];
-      if (numMachineSet > 0) {
-        numNodes = 0;
-        for (Iterator<DatanodeDescriptor> it = 
-             blocksMap.nodeIterator(blocks[curBlk]); it.hasNext();) {
-          DatanodeDescriptor dn = it.next();
-          boolean replicaCorrupt = 
-            corruptReplicas.isReplicaCorrupt(blocks[curBlk], dn);
-          if (blockCorrupt || (!blockCorrupt && !replicaCorrupt))
-            machineSet[numNodes++] = dn;
-        }
-      }
-      LocatedBlock b = new LocatedBlock(blocks[curBlk], machineSet, curPos,
-          blockCorrupt);
-      if (namesystem.isAccessTokenEnabled) {
-        b.setAccessToken(namesystem.accessTokenHandler.generateToken(b.getBlock()
-            .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.READ)));
-      }
-      results.add(b);
+      results.add(getBlockLocation(blocks[curBlk], curPos));
       curPos += blocks[curBlk].getNumBytes();
       curPos += blocks[curBlk].getNumBytes();
       curBlk++;
       curBlk++;
     } while (curPos < endOff 
     } while (curPos < endOff 
@@ -314,6 +358,41 @@ public class BlockManager {
     return results;
     return results;
   }
   }
 
 
+  /** @return a LocatedBlock for the given block */
+  LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
+      ) throws IOException {
+    if (!blk.isComplete()) {
+      final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
+      final DatanodeDescriptor[] locations = uc.getExpectedLocations();
+      return namesystem.createLocatedBlock(uc, locations, pos, false);
+    }
+
+    // get block locations
+    final int numCorruptNodes = countNodes(blk).corruptReplicas();
+    final int numCorruptReplicas = corruptReplicas.numCorruptReplicas(blk);
+    if (numCorruptNodes != numCorruptReplicas) {
+      FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for "
+          + blk + " blockMap has " + numCorruptNodes
+          + " but corrupt replicas map has " + numCorruptReplicas);
+    }
+
+    final int numNodes = blocksMap.numNodes(blk);
+    final boolean isCorrupt = numCorruptNodes == numNodes;
+    final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
+    final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines];
+    if (numMachines > 0) {
+      int j = 0;
+      for(Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(blk);
+          it.hasNext();) {
+        final DatanodeDescriptor d = it.next();
+        final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
+        if (isCorrupt || (!isCorrupt && !replicaCorrupt))
+          machines[j++] = d;
+      }
+    }
+    return namesystem.createLocatedBlock(blk, machines, pos, isCorrupt);    
+  }
+
   /**
   /**
    * Check whether the replication parameter is within the range
    * Check whether the replication parameter is within the range
    * determined by system configuration.
    * determined by system configuration.
@@ -369,7 +448,7 @@ public class BlockManager {
       pendingDeletionBlocksCount++;
       pendingDeletionBlocksCount++;
       if (log) {
       if (log) {
         NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
         NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
-            + b.getBlockName() + " to " + dn.getName());
+            + b + " to " + dn.getName());
       }
       }
     }
     }
   }
   }
@@ -399,7 +478,7 @@ public class BlockManager {
     }
     }
     if (datanodes.length() != 0) {
     if (datanodes.length() != 0) {
       NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
       NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
-          + b.getBlockName() + " to " + datanodes.toString());
+          + b + " to " + datanodes.toString());
     }
     }
   }
   }
 
 
@@ -457,6 +536,10 @@ public class BlockManager {
       addToInvalidates(storedBlock, node);
       addToInvalidates(storedBlock, node);
       return;
       return;
     } 
     } 
+
+    // Add replica to the data-node if it is not already there
+    node.addBlock(storedBlock);
+
     // Add this replica to corruptReplicas Map
     // Add this replica to corruptReplicas Map
     corruptReplicas.addToCorruptReplicasMap(storedBlock, node);
     corruptReplicas.addToCorruptReplicasMap(storedBlock, node);
     if (countNodes(storedBlock).liveReplicas() > inode.getReplication()) {
     if (countNodes(storedBlock).liveReplicas() > inode.getReplication()) {
@@ -885,7 +968,8 @@ public class BlockManager {
     Collection<Block> toAdd = new LinkedList<Block>();
     Collection<Block> toAdd = new LinkedList<Block>();
     Collection<Block> toRemove = new LinkedList<Block>();
     Collection<Block> toRemove = new LinkedList<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
-    node.reportDiff(blocksMap, report, toAdd, toRemove, toInvalidate);
+    Collection<BlockInfo> toCorrupt = new LinkedList<BlockInfo>();
+    node.reportDiff(this, report, toAdd, toRemove, toInvalidate, toCorrupt);
 
 
     for (Block b : toRemove) {
     for (Block b : toRemove) {
       removeStoredBlock(b, node);
       removeStoredBlock(b, node);
@@ -899,6 +983,9 @@ public class BlockManager {
           + " does not belong to any file.");
           + " does not belong to any file.");
       addToInvalidates(b, node);
       addToInvalidates(b, node);
     }
     }
+    for (BlockInfo b : toCorrupt) {
+      markBlockAsCorrupt(b, node);
+    }
   }
   }
 
 
   /**
   /**
@@ -908,7 +995,8 @@ public class BlockManager {
    */
    */
   private Block addStoredBlock(final Block block,
   private Block addStoredBlock(final Block block,
                                DatanodeDescriptor node,
                                DatanodeDescriptor node,
-                               DatanodeDescriptor delNodeHint) {
+                               DatanodeDescriptor delNodeHint)
+  throws IOException {
     BlockInfo storedBlock = blocksMap.getStoredBlock(block);
     BlockInfo storedBlock = blocksMap.getStoredBlock(block);
     if (storedBlock == null || storedBlock.getINode() == null) {
     if (storedBlock == null || storedBlock.getINode() == null) {
       // If this block does not belong to anyfile, then we are done.
       // If this block does not belong to anyfile, then we are done.
@@ -1020,13 +1108,17 @@ public class BlockManager {
     int numCurrentReplica = numLiveReplicas
     int numCurrentReplica = numLiveReplicas
       + pendingReplications.getNumReplicas(storedBlock);
       + pendingReplications.getNumReplicas(storedBlock);
 
 
+    if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
+        numLiveReplicas >= minReplication)
+      storedBlock = completeBlock(fileINode, storedBlock);
+
     // check whether safe replication is reached for the block
     // check whether safe replication is reached for the block
-    namesystem.incrementSafeBlockCount(numCurrentReplica);
+    // only complete blocks are counted towards that
+    if(storedBlock.isComplete())
+      namesystem.incrementSafeBlockCount(numCurrentReplica);
 
 
-    //
-    // if file is being actively written to, then do not check
-    // replication-factor here. It will be checked when the file is closed.
-    //
+    // if file is under construction, then check whether the block
+    // can be completed
     if (fileINode.isUnderConstruction()) {
     if (fileINode.isUnderConstruction()) {
       return storedBlock;
       return storedBlock;
     }
     }
@@ -1253,7 +1345,30 @@ public class BlockManager {
     // Modify the blocks->datanode map and node's map.
     // Modify the blocks->datanode map and node's map.
     //
     //
     pendingReplications.remove(block);
     pendingReplications.remove(block);
-    addStoredBlock(block, node, delHintNode);
+
+    // blockReceived reports a finalized block
+    Collection<Block> toAdd = new LinkedList<Block>();
+    Collection<Block> toInvalidate = new LinkedList<Block>();
+    Collection<BlockInfo> toCorrupt = new LinkedList<BlockInfo>();
+    node.processReportedBlock(this, block, ReplicaState.FINALIZED,
+                              toAdd, toInvalidate, toCorrupt);
+    // the block is only in one of the lists
+    // if it is in none then data-node already has it
+    assert toAdd.size() + toInvalidate.size() <= 1 :
+      "The block should be only in one of the lists.";
+
+    for (Block b : toAdd) {
+      addStoredBlock(b, node, delHintNode);
+    }
+    for (Block b : toInvalidate) {
+      NameNode.stateChangeLog.info("BLOCK* NameSystem.addBlock: block "
+          + b + " on " + node.getName() + " size " + b.getNumBytes()
+          + " does not belong to any file.");
+      addToInvalidates(b, node);
+    }
+    for (BlockInfo b : toCorrupt) {
+      markBlockAsCorrupt(b, node);
+    }
   }
   }
 
 
   /**
   /**
@@ -1351,6 +1466,14 @@ public class BlockManager {
     return blocksMap.getStoredBlock(block);
     return blocksMap.getStoredBlock(block);
   }
   }
 
 
+  /**
+   * Find the block by block ID.
+   */
+  BlockInfo findStoredBlock(long blockId) {
+    Block wildcardBlock = new Block(blockId, 0, GenerationStamp.WILDCARD_STAMP);
+    return blocksMap.getStoredBlock(wildcardBlock);
+  }
+
   /* updates a block in under replication queue */
   /* updates a block in under replication queue */
   void updateNeededReplications(Block block, int curReplicasDelta,
   void updateNeededReplications(Block block, int curReplicasDelta,
       int expectedReplicasDelta) {
       int expectedReplicasDelta) {
@@ -1522,7 +1645,7 @@ public class BlockManager {
     return Math.max(missingBlocksInPrevIter, missingBlocksInCurIter);
     return Math.max(missingBlocksInPrevIter, missingBlocksInCurIter);
   }
   }
 
 
-  BlockInfo addINode(Block block, INodeFile iNode) {
+  BlockInfo addINode(BlockInfo block, INodeFile iNode) {
     return blocksMap.addINode(block, iNode);
     return blocksMap.addINode(block, iNode);
   }
   }
 
 

+ 22 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -75,11 +75,10 @@ class BlocksMap {
   /**
   /**
    * Add block b belonging to the specified file inode to the map.
    * Add block b belonging to the specified file inode to the map.
    */
    */
-  BlockInfo addINode(Block b, INodeFile iNode) {
-    int replication = iNode.getReplication();
+  BlockInfo addINode(BlockInfo b, INodeFile iNode) {
     BlockInfo info = map.get(b);
     BlockInfo info = map.get(b);
-    if (info == null) {
-      info = new BlockInfo(b, replication);
+    if (info != b) {
+      info = b;
       map.put(info, info);
       map.put(info, info);
     }
     }
     info.setINode(iNode);
     info.setINode(iNode);
@@ -191,4 +190,23 @@ class BlocksMap {
   float getLoadFactor() {
   float getLoadFactor() {
     return loadFactor;
     return loadFactor;
   }
   }
+
+  /**
+   * Replace a block in the block map by a new block.
+   * The new block and the old one have the same key.
+   * @param newBlock - block for replacement
+   * @return new block
+   */
+  BlockInfo replaceBlock(BlockInfo newBlock) {
+    BlockInfo currentBlock = map.get(newBlock);
+    assert currentBlock != null : "the block if not in blocksMap";
+    // replace block in data-node lists
+    for(int idx = currentBlock.numNodes()-1; idx >= 0; idx--) {
+      DatanodeDescriptor dn = currentBlock.getDatanode(idx);
+      dn.replaceBlock(currentBlock, newBlock);
+    }
+    // replace block in the map itself
+    map.put(newBlock, newBlock);
+    return newBlock;
+  }
 }
 }

+ 165 - 40
src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -25,7 +25,11 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
@@ -55,29 +59,36 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
   }
 
 
   /** A BlockTargetPair queue. */
   /** A BlockTargetPair queue. */
-  private static class BlockQueue {
-    private final Queue<BlockTargetPair> blockq = new LinkedList<BlockTargetPair>();
+  private static class BlockQueue<E> {
+    private final Queue<E> blockq = new LinkedList<E>();
 
 
     /** Size of the queue */
     /** Size of the queue */
     synchronized int size() {return blockq.size();}
     synchronized int size() {return blockq.size();}
 
 
     /** Enqueue */
     /** Enqueue */
-    synchronized boolean offer(Block block, DatanodeDescriptor[] targets) { 
-      return blockq.offer(new BlockTargetPair(block, targets));
+    synchronized boolean offer(E e) { 
+      return blockq.offer(e);
     }
     }
 
 
     /** Dequeue */
     /** Dequeue */
-    synchronized List<BlockTargetPair> poll(int numBlocks) {
+    synchronized List<E> poll(int numBlocks) {
       if (numBlocks <= 0 || blockq.isEmpty()) {
       if (numBlocks <= 0 || blockq.isEmpty()) {
         return null;
         return null;
       }
       }
 
 
-      List<BlockTargetPair> results = new ArrayList<BlockTargetPair>();
+      List<E> results = new ArrayList<E>();
       for(; !blockq.isEmpty() && numBlocks > 0; numBlocks--) {
       for(; !blockq.isEmpty() && numBlocks > 0; numBlocks--) {
         results.add(blockq.poll());
         results.add(blockq.poll());
       }
       }
       return results;
       return results;
     }
     }
+
+    /**
+     * Returns <tt>true</tt> if the queue contains the specified element.
+     */
+    boolean contains(E e) {
+      return blockq.contains(e);
+    }
   }
   }
 
 
   private volatile BlockInfo blockList = null;
   private volatile BlockInfo blockList = null;
@@ -87,9 +98,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
   protected boolean needKeyUpdate = false;
   protected boolean needKeyUpdate = false;
 
 
   /** A queue of blocks to be replicated by this datanode */
   /** A queue of blocks to be replicated by this datanode */
-  private BlockQueue replicateBlocks = new BlockQueue();
+  private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
   /** A queue of blocks to be recovered by this datanode */
   /** A queue of blocks to be recovered by this datanode */
-  private BlockQueue recoverBlocks = new BlockQueue();
+  private BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+                                new BlockQueue<BlockInfoUnderConstruction>();
   /** 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>();
 
 
@@ -201,6 +213,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
     blockList = b.listInsert(blockList, this);
     blockList = b.listInsert(blockList, this);
   }
   }
 
 
+  /**
+   * Replace specified old block with a new one in the DataNodeDescriptor.
+   * 
+   * @param oldBlock - block to be replaced
+   * @param newBlock - a replacement block
+   * @return the new block
+   */
+  BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
+    boolean done = removeBlock(oldBlock);
+    assert done : "Old block should belong to the data-node when replacing";
+    done = addBlock(newBlock);
+    assert done : "New block should not belong to the data-node when replacing";
+    return newBlock;
+  }
+
   void resetBlocks() {
   void resetBlocks() {
     this.capacity = 0;
     this.capacity = 0;
     this.remaining = 0;
     this.remaining = 0;
@@ -262,15 +289,20 @@ public class DatanodeDescriptor extends DatanodeInfo {
    */
    */
   void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) {
   void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) {
     assert(block != null && targets != null && targets.length > 0);
     assert(block != null && targets != null && targets.length > 0);
-    replicateBlocks.offer(block, targets);
+    replicateBlocks.offer(new BlockTargetPair(block, targets));
   }
   }
 
 
   /**
   /**
    * Store block recovery work.
    * Store block recovery work.
    */
    */
-  void addBlockToBeRecovered(Block block, DatanodeDescriptor[] targets) {
-    assert(block != null && targets != null && targets.length > 0);
-    recoverBlocks.offer(block, targets);
+  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
+    if(recoverBlocks.contains(block)) {
+      // this prevents adding the same block twice to the recovery queue
+      FSNamesystem.LOG.info("Block " + block +
+                            " is already in the recovery queue.");
+      return;
+    }
+    recoverBlocks.offer(block);
   }
   }
 
 
   /**
   /**
@@ -308,10 +340,16 @@ public class DatanodeDescriptor extends DatanodeInfo {
         new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blocktargetlist);
         new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blocktargetlist);
   }
   }
 
 
-  BlockCommand getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockTargetPair> blocktargetlist = recoverBlocks.poll(maxTransfers);
-    return blocktargetlist == null? null:
-        new BlockCommand(DatanodeProtocol.DNA_RECOVERBLOCK, blocktargetlist);
+  BlockRecoveryCommand getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+    if(blocks == null)
+      return null;
+    BlockRecoveryCommand brCommand = new BlockRecoveryCommand(blocks.size());
+    for(BlockInfoUnderConstruction b : blocks) {
+      brCommand.add(new RecoveringBlock(
+          b, b.getExpectedLocations(), b.getBlockRecoveryId()));
+    }
+    return brCommand;
   }
   }
 
 
   /**
   /**
@@ -361,11 +399,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return blockarray;
     return blockarray;
   }
   }
 
 
-  void reportDiff(BlocksMap blocksMap,
+  void reportDiff(BlockManager blockManager,
                   BlockListAsLongs newReport,
                   BlockListAsLongs newReport,
                   Collection<Block> toAdd,    // add to DatanodeDescriptor
                   Collection<Block> toAdd,    // add to DatanodeDescriptor
                   Collection<Block> toRemove, // remove from DatanodeDescriptor
                   Collection<Block> toRemove, // remove from DatanodeDescriptor
-                  Collection<Block> toInvalidate) { // should be removed from DN
+                  Collection<Block> toInvalidate, // should be removed from DN
+                  Collection<BlockInfo> toCorrupt) {// add to corrupt replicas
     // place a delimiter in the list which separates blocks 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
     // that have been reported from those that have not
     BlockInfo delimiter = new BlockInfo(new Block(), 1);
     BlockInfo delimiter = new BlockInfo(new Block(), 1);
@@ -373,29 +412,16 @@ public class DatanodeDescriptor extends DatanodeInfo {
     assert added : "Delimiting block cannot be present in the node";
     assert added : "Delimiting block cannot be present in the node";
     if(newReport == null)
     if(newReport == null)
       newReport = new BlockListAsLongs();
       newReport = new BlockListAsLongs();
-    // scan the report and collect newly reported blocks
-    // Note we are taking special precaution to limit tmp blocks allocated
-    // as part this block report - which why block list is stored as longs
-    for (Block iblk : newReport) {
-      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
-      if(storedBlock == null) {
-        // If block is not in blocksMap it does not belong to any file
-        toInvalidate.add(new Block(iblk));
-        continue;
-      }
-      if(storedBlock.findDatanode(this) < 0) {// Known block, but not on the DN
-        // if the size differs from what is in the blockmap, then return
-        // the new block. addStoredBlock will then pick up the right size of this
-        // block and will update the block object in the BlocksMap
-        if (storedBlock.getNumBytes() != iblk.getNumBytes()) {
-          toAdd.add(new Block(iblk));
-        } else {
-          toAdd.add(storedBlock);
-        }
-        continue;
-      }
+    // scan the report and process newly reported blocks
+    BlockReportIterator itBR = newReport.getBlockReportIterator();
+    while(itBR.hasNext()) {
+      Block iblk = itBR.next();
+      ReplicaState iState = itBR.getCurrentReplicaState();
+      BlockInfo storedBlock = processReportedBlock(blockManager, iblk, iState,
+                                               toAdd, toInvalidate, toCorrupt);
       // move block to the head of the list
       // move block to the head of the list
-      this.moveBlockToHead(storedBlock);
+      if(storedBlock != null && storedBlock.findDatanode(this) >= 0)
+        this.moveBlockToHead(storedBlock);
     }
     }
     // collect blocks that have not been reported
     // collect blocks that have not been reported
     // all of them are next to the delimiter
     // all of them are next to the delimiter
@@ -405,6 +431,105 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.removeBlock(delimiter);
     this.removeBlock(delimiter);
   }
   }
 
 
+  /**
+   * Process a block replica reported by the data-node.
+   * 
+   * <ol>
+   * <li>If the block is not known to the system (not in blocksMap) then the
+   * data-node should be notified to invalidate this block.</li>
+   * <li>If the reported replica is valid that is has the same generation stamp
+   * and length as recorded on the name-node, then the replica location is
+   * added to the name-node.</li>
+   * <li>If the reported replica is not valid, then it is marked as corrupt,
+   * which triggers replication of the existing valid replicas.
+   * Corrupt replicas are removed from the system when the block
+   * is fully replicated.</li>
+   * </ol>
+   * 
+   * @param blockManager
+   * @param block reported block replica
+   * @param rState reported replica state
+   * @param toAdd add to DatanodeDescriptor
+   * @param toInvalidate missing blocks (not in the blocks map)
+   *        should be removed from the data-node
+   * @param toCorrupt replicas with unexpected length or generation stamp;
+   *        add to corrupt replicas
+   * @return
+   */
+  BlockInfo processReportedBlock(
+                  BlockManager blockManager,
+                  Block block,                // reported block replica
+                  ReplicaState rState,        // reported replica state
+                  Collection<Block> toAdd,    // add to DatanodeDescriptor
+                  Collection<Block> toInvalidate, // should be removed from DN
+                  Collection<BlockInfo> toCorrupt) {// add to corrupt replicas
+    FSNamesystem.LOG.debug("Reported block " + block
+        + " on " + getName() + " size " + block.getNumBytes()
+        + " replicaState = " + rState);
+
+    // find block by blockId
+    BlockInfo storedBlock = blockManager.findStoredBlock(block.getBlockId());
+    if(storedBlock == null) {
+      // If blocksMap does not contain reported block id,
+      // the replica should be removed from the data-node.
+      toInvalidate.add(new Block(block));
+      return null;
+    }
+
+    FSNamesystem.LOG.debug("In memory blockUCState = " + storedBlock.getBlockUCState());
+
+    // Block is on the DN
+    boolean isCorrupt = false;
+    switch(rState) {
+    case FINALIZED:
+      switch(storedBlock.getBlockUCState()) {
+      case COMPLETE:
+      case COMMITTED:
+        // This is a temporary hack until Block.equals() and compareTo()
+        // are changed not to take into account the generation stamp for searching
+        // in  blocksMap
+        if(storedBlock.getGenerationStamp() != block.getGenerationStamp()) {
+          toInvalidate.add(new Block(block));
+          return storedBlock;
+        }
+
+        if(storedBlock.getGenerationStamp() != block.getGenerationStamp()
+            || storedBlock.getNumBytes() != block.getNumBytes())
+          isCorrupt = true;
+        break;
+      case UNDER_CONSTRUCTION:
+      case UNDER_RECOVERY:
+        ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
+            this, block, rState);
+      }
+      if(!isCorrupt && storedBlock.findDatanode(this) < 0)
+        if (storedBlock.getNumBytes() != block.getNumBytes()) {
+          toAdd.add(new Block(block));
+        } else {
+          toAdd.add(storedBlock);
+        }
+      break;
+    case RBW:
+    case RWR:
+      if(!storedBlock.isComplete())
+        ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
+                                                      this, block, rState);
+      else
+        isCorrupt = true;
+      break;
+    case RUR:       // should not be reported
+    case TEMPORARY: // should not be reported
+    default:
+      FSNamesystem.LOG.warn("Unexpected replica state " + rState
+          + " for block: " + storedBlock + 
+          " on " + getName() + " size " + storedBlock.getNumBytes());
+      break;
+    }
+    if(isCorrupt)
+        toCorrupt.add(storedBlock);
+    return storedBlock;
+  }
+
   /** Serialization for FSEditLog */
   /** Serialization for FSEditLog */
   void readFieldsFromFSEditLog(DataInput in) throws IOException {
   void readFieldsFromFSEditLog(DataInput in) throws IOException {
     this.name = DeprecatedUTF8.readString(in);
     this.name = DeprecatedUTF8.readString(in);

+ 27 - 13
src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsRecord;
@@ -191,7 +192,7 @@ class FSDirectory implements Closeable {
    */
    */
   INode unprotectedAddFile( String path, 
   INode unprotectedAddFile( String path, 
                             PermissionStatus permissions,
                             PermissionStatus permissions,
-                            Block[] blocks, 
+                            BlockInfo[] blocks, 
                             short replication,
                             short replication,
                             long modificationTime,
                             long modificationTime,
                             long atime,
                             long atime,
@@ -261,7 +262,8 @@ class FSDirectory implements Closeable {
         // Add file->block mapping
         // Add file->block mapping
         INodeFile newF = (INodeFile)newNode;
         INodeFile newF = (INodeFile)newNode;
         for (int i = 0; i < nrBlocks; i++) {
         for (int i = 0; i < nrBlocks; i++) {
-          newF.setBlock(i, getBlockManager().addINode(blocks[i], newF));
+          BlockInfo blockInfo = new BlockInfo(blocks[i], newF.getReplication());
+          newF.setBlock(i, getBlockManager().addINode(blockInfo, newF));
         }
         }
       }
       }
     }
     }
@@ -271,27 +273,39 @@ class FSDirectory implements Closeable {
   /**
   /**
    * Add a block to the file. Returns a reference to the added block.
    * Add a block to the file. Returns a reference to the added block.
    */
    */
-  Block addBlock(String path, INode[] inodes, Block block
-      ) throws QuotaExceededException  {
+  BlockInfo addBlock(String path,
+                     INode[] inodes,
+                     Block block,
+                     DatanodeDescriptor targets[]
+  ) throws QuotaExceededException, IOException  {
     waitForReady();
     waitForReady();
 
 
     synchronized (rootDir) {
     synchronized (rootDir) {
-      INodeFile fileNode = (INodeFile) inodes[inodes.length-1];
+      assert inodes[inodes.length-1].isUnderConstruction() :
+        "INode should correspond to a file under construction";
+      INodeFileUnderConstruction fileINode = 
+        (INodeFileUnderConstruction)inodes[inodes.length-1];
 
 
       // check quota limits and updated space consumed
       // check quota limits and updated space consumed
       updateCount(inodes, inodes.length-1, 0, 
       updateCount(inodes, inodes.length-1, 0, 
-                  fileNode.getPreferredBlockSize()*fileNode.getReplication());
-      
-      // associate the new list of blocks with this file
-      BlockInfo blockInfo = getBlockManager().addINode(block, fileNode);
-      fileNode.addBlock(blockInfo);
+                  fileINode.getPreferredBlockSize()*fileINode.getReplication());
+
+      // associate new last block for the file
+      BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstruction(
+            block,
+            fileINode.getReplication(),
+            BlockUCState.UNDER_CONSTRUCTION,
+            targets);
+      getBlockManager().addINode(blockInfo, fileINode);
+      fileINode.addBlock(blockInfo);
 
 
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
                                     + path + " with " + block
                                     + path + " with " + block
                                     + " block is added to the in-memory "
                                     + " block is added to the in-memory "
                                     + "file system");
                                     + "file system");
+      return blockInfo;
     }
     }
-    return block;
   }
   }
 
 
   /**
   /**
@@ -335,7 +349,7 @@ class FSDirectory implements Closeable {
 
 
     synchronized (rootDir) {
     synchronized (rootDir) {
       // modify file-> block and blocksMap
       // modify file-> block and blocksMap
-      fileNode.removeBlock(block);
+      fileNode.removeLastBlock(block);
       getBlockManager().removeBlockFromMap(block);
       getBlockManager().removeBlockFromMap(block);
       // If block is removed from blocksMap remove it from corruptReplicasMap
       // If block is removed from blocksMap remove it from corruptReplicasMap
       getBlockManager().removeFromCorruptReplicasMap(block);
       getBlockManager().removeFromCorruptReplicasMap(block);
@@ -732,7 +746,7 @@ class FSDirectory implements Closeable {
       }
       }
       
       
       int index = 0;
       int index = 0;
-      for (Block b : newnode.getBlocks()) {
+      for (BlockInfo b : newnode.getBlocks()) {
         BlockInfo info = getBlockManager().addINode(b, newnode);
         BlockInfo info = getBlockManager().addINode(b, newnode);
         newnode.setBlock(index, info); // inode refers to the block in BlocksMap
         newnode.setBlock(index, info); // inode refers to the block in BlocksMap
         index++;
         index++;

+ 23 - 19
src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -461,19 +461,9 @@ public class FSEditLog {
             blockSize = readLong(in);
             blockSize = readLong(in);
           }
           }
           // get blocks
           // get blocks
-          Block blocks[] = null;
-          if (logVersion <= -14) {
-            blocks = readBlocks(in);
-          } else {
-            BlockTwo oldblk = new BlockTwo();
-            int num = in.readInt();
-            blocks = new Block[num];
-            for (int i = 0; i < num; i++) {
-              oldblk.readFields(in);
-              blocks[i] = new Block(oldblk.blkid, oldblk.len, 
-                                    Block.GRANDFATHER_GENERATION_STAMP);
-            }
-          }
+          boolean isFileUnderConstruction = (opcode == OP_ADD);
+          BlockInfo blocks[] = 
+            readBlocks(in, logVersion, isFileUnderConstruction, replication);
 
 
           // Older versions of HDFS does not store the block size in inode.
           // Older versions of HDFS does not store the block size in inode.
           // If the file has more than one block, use the size of the
           // If the file has more than one block, use the size of the
@@ -521,7 +511,7 @@ public class FSEditLog {
                                                     path, permissions,
                                                     path, permissions,
                                                     blocks, replication, 
                                                     blocks, replication, 
                                                     mtime, atime, blockSize);
                                                     mtime, atime, blockSize);
-          if (opcode == OP_ADD) {
+          if (isFileUnderConstruction) {
             numOpAdd++;
             numOpAdd++;
             //
             //
             // Replace current node with a INodeUnderConstruction.
             // Replace current node with a INodeUnderConstruction.
@@ -538,7 +528,7 @@ public class FSEditLog {
                                       clientMachine, 
                                       clientMachine, 
                                       null);
                                       null);
             fsDir.replaceNode(path, node, cons);
             fsDir.replaceNode(path, node, cons);
-            fsNamesys.leaseManager.addLease(cons.clientName, path);
+            fsNamesys.leaseManager.addLease(cons.getClientName(), path);
           }
           }
           break;
           break;
         } 
         } 
@@ -1247,12 +1237,26 @@ public class FSEditLog {
     return Long.parseLong(FSImage.readString(in));
     return Long.parseLong(FSImage.readString(in));
   }
   }
 
 
-  static private Block[] readBlocks(DataInputStream in) throws IOException {
+  static private BlockInfo[] readBlocks(
+      DataInputStream in,
+      int logVersion,
+      boolean isFileUnderConstruction,
+      short replication) throws IOException {
     int numBlocks = in.readInt();
     int numBlocks = in.readInt();
-    Block[] blocks = new Block[numBlocks];
+    BlockInfo[] blocks = new BlockInfo[numBlocks];
+    Block blk = new Block();
+    BlockTwo oldblk = new BlockTwo();
     for (int i = 0; i < numBlocks; i++) {
     for (int i = 0; i < numBlocks; i++) {
-      blocks[i] = new Block();
-      blocks[i].readFields(in);
+      if (logVersion <= -14) {
+        blk.readFields(in);
+      } else {
+        oldblk.readFields(in);
+        blk.set(oldblk.blkid, oldblk.len, Block.GRANDFATHER_GENERATION_STAMP);
+      }
+      if(isFileUnderConstruction && i == numBlocks-1)
+        blocks[i] = new BlockInfoUnderConstruction(blk, replication);
+      else
+        blocks[i] = new BlockInfo(blk, replication);
     }
     }
     return blocks;
     return blocks;
   }
   }

+ 11 - 3
src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.UpgradeManager;
 import org.apache.hadoop.hdfs.server.common.UpgradeManager;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
@@ -1403,7 +1404,7 @@ public class FSImage extends Storage {
       }
       }
       INodeFile oldnode = (INodeFile) old;
       INodeFile oldnode = (INodeFile) old;
       fsDir.replaceNode(path, oldnode, cons);
       fsDir.replaceNode(path, oldnode, cons);
-      fs.leaseManager.addLease(cons.clientName, path); 
+      fs.leaseManager.addLease(cons.getClientName(), path); 
     }
     }
   }
   }
 
 
@@ -1419,10 +1420,17 @@ public class FSImage extends Storage {
     int numBlocks = in.readInt();
     int numBlocks = in.readInt();
     BlockInfo[] blocks = new BlockInfo[numBlocks];
     BlockInfo[] blocks = new BlockInfo[numBlocks];
     Block blk = new Block();
     Block blk = new Block();
-    for (int i = 0; i < numBlocks; i++) {
+    int i = 0;
+    for (; i < numBlocks-1; i++) {
       blk.readFields(in);
       blk.readFields(in);
       blocks[i] = new BlockInfo(blk, blockReplication);
       blocks[i] = new BlockInfo(blk, blockReplication);
     }
     }
+    // last block is UNDER_CONSTRUCTION
+    if(numBlocks > 0) {
+      blk.readFields(in);
+      blocks[i] = new BlockInfoUnderConstruction(
+        blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+    }
     PermissionStatus perm = PermissionStatus.read(in);
     PermissionStatus perm = PermissionStatus.read(in);
     String clientName = readString(in);
     String clientName = readString(in);
     String clientMachine = readString(in);
     String clientMachine = readString(in);
@@ -1430,7 +1438,7 @@ public class FSImage extends Storage {
     // These locations are not used at all
     // These locations are not used at all
     int numLocs = in.readInt();
     int numLocs = in.readInt();
     DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocs];
     DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocs];
-    for (int i = 0; i < numLocs; i++) {
+    for (i = 0; i < numLocs; i++) {
       locations[i] = new DatanodeDescriptor();
       locations[i] = new DatanodeDescriptor();
       locations[i].readFields(in);
       locations[i].readFields(in);
     }
     }

+ 402 - 112
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@@ -563,13 +564,18 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
     }
     List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
     List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
     long totalSize = 0;
     long totalSize = 0;
+    BlockInfo curBlock;
     while(totalSize<size && iter.hasNext()) {
     while(totalSize<size && iter.hasNext()) {
-      totalSize += addBlock(iter.next(), results);
+      curBlock = iter.next();
+      if(!curBlock.isComplete())  continue;
+      totalSize += addBlock(curBlock, results);
     }
     }
     if(totalSize<size) {
     if(totalSize<size) {
       iter = node.getBlockIterator(); // start from the beginning
       iter = node.getBlockIterator(); // start from the beginning
       for(int i=0; i<startBlock&&totalSize<size; i++) {
       for(int i=0; i<startBlock&&totalSize<size; i++) {
-        totalSize += addBlock(iter.next(), results);
+        curBlock = iter.next();
+        if(!curBlock.isComplete())  continue;
+        totalSize += addBlock(curBlock, results);
       }
       }
     }
     }
 
 
@@ -707,14 +713,46 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (doAccessTime && isAccessTimeSupported()) {
     if (doAccessTime && isAccessTimeSupported()) {
       dir.setTimes(src, inode, -1, now(), false);
       dir.setTimes(src, inode, -1, now(), false);
     }
     }
-    final Block[] blocks = inode.getBlocks();
+    final BlockInfo[] blocks = inode.getBlocks();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("blocks = " + java.util.Arrays.asList(blocks));
+    }
     if (blocks == null) {
     if (blocks == null) {
       return null;
       return null;
     }
     }
-    final List<LocatedBlock> results = blocks.length == 0?
-        new ArrayList<LocatedBlock>(0):
-        blockManager.getBlockLocations(blocks, offset, length, Integer.MAX_VALUE);
-    return inode.createLocatedBlocks(results);
+
+    if (blocks.length == 0) {
+      return new LocatedBlocks(0, inode.isUnderConstruction(),
+          Collections.<LocatedBlock>emptyList(), null, false);
+    } else {
+      final long n = inode.computeFileSize(false);
+      final List<LocatedBlock> locatedblocks = blockManager.getBlockLocations(
+          blocks, offset, length, Integer.MAX_VALUE);
+      final BlockInfo last = inode.getLastBlock();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("last = " + last);
+      }
+
+      if (!last.isComplete()) {
+        return new LocatedBlocks(n, inode.isUnderConstruction(), locatedblocks,
+            blockManager.getBlockLocation(last, n), false);
+      }
+      else {
+        return new LocatedBlocks(n, inode.isUnderConstruction(), locatedblocks,
+            blockManager.getBlockLocation(last, n-last.getNumBytes()), true);
+      }
+    }
+  }
+
+  /** Create a LocatedBlock. */
+  LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
+      final long offset, final boolean corrupt) throws IOException {
+    final LocatedBlock lb = new LocatedBlock(b, locations, offset, corrupt);
+    if (isAccessTokenEnabled) {
+      lb.setAccessToken(accessTokenHandler.generateToken(b.getBlockId(),
+          EnumSet.of(AccessTokenHandler.AccessMode.READ)));
+    }
+    return lb;
   }
   }
 
 
   /**
   /**
@@ -912,40 +950,45 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         // If the file is under construction , then it must be in our
         // If the file is under construction , then it must be in our
         // leases. Find the appropriate lease record.
         // leases. Find the appropriate lease record.
         //
         //
-        Lease lease = leaseManager.getLease(holder);
-        //
-        // We found the lease for this file. And surprisingly the original
-        // holder is trying to recreate this file. This should never occur.
-        //
-        if (lease != null) {
+        Lease lease = leaseManager.getLeaseByPath(src);
+        if (lease == null) {
           throw new AlreadyBeingCreatedException(
           throw new AlreadyBeingCreatedException(
-                                                 "failed to create file " + src + " for " + holder +
-                                                 " on client " + clientMachine + 
-                                                 " because current leaseholder is trying to recreate file.");
+              "failed to create file " + src + " for " + holder +
+              " on client " + clientMachine + 
+              " because pendingCreates is non-null but no leases found.");
         }
         }
         //
         //
-        // Find the original holder.
+        // We found the lease for this file. And surprisingly the original
+        // holder is trying to recreate this file. This should never occur.
         //
         //
-        lease = leaseManager.getLease(pendingFile.clientName);
-        if (lease == null) {
+        if (lease.getHolder().equals(holder)) {
           throw new AlreadyBeingCreatedException(
           throw new AlreadyBeingCreatedException(
-                                                 "failed to create file " + src + " for " + holder +
-                                                 " on client " + clientMachine + 
-                                                 " because pendingCreates is non-null but no leases found.");
+              "failed to create file " + src + " for " + holder +
+              " on client " + clientMachine + 
+              " because current leaseholder is trying to recreate file.");
         }
         }
+        assert lease.getHolder().equals(pendingFile.getClientName()) :
+          "Current lease holder " + lease.getHolder() +
+          " does not match file creator " + pendingFile.getClientName();
         //
         //
+        // Current lease holder is different from the requester.
         // If the original holder has not renewed in the last SOFTLIMIT 
         // If the original holder has not renewed in the last SOFTLIMIT 
-        // period, then start lease recovery.
+        // period, then start lease recovery, otherwise fail.
         //
         //
         if (lease.expiredSoftLimit()) {
         if (lease.expiredSoftLimit()) {
           LOG.info("startFile: recover lease " + lease + ", src=" + src);
           LOG.info("startFile: recover lease " + lease + ", src=" + src);
-          internalReleaseLease(lease, src);
-        }
-        throw new AlreadyBeingCreatedException("failed to create file " + src + " for " + holder +
-                                               " on client " + clientMachine + 
-                                               ", because this file is already being created by " +
-                                               pendingFile.getClientName() + 
-                                               " on " + pendingFile.getClientMachine());
+          boolean isClosed = internalReleaseLease(lease, src, null);
+          if(!isClosed)
+            throw new RecoveryInProgressException(
+                "Failed to close file " + src +
+                ". Lease recovery is in progress. Try again later.");
+
+        } else
+          throw new AlreadyBeingCreatedException("failed to create file " +
+              src + " for " + holder + " on client " + clientMachine + 
+              ", because this file is already being created by " +
+              pendingFile.getClientName() + 
+              " on " + pendingFile.getClientMachine());
       }
       }
 
 
       try {
       try {
@@ -998,7 +1041,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                         clientMachine,
                                         clientMachine,
                                         clientNode);
                                         clientNode);
         dir.replaceNode(src, node, cons);
         dir.replaceNode(src, node, cons);
-        leaseManager.addLease(cons.clientName, src);
+        leaseManager.addLease(cons.getClientName(), src);
 
 
       } else {
       } else {
        // Now we can add the name to the filesystem. This file has no
        // Now we can add the name to the filesystem. This file has no
@@ -1014,7 +1057,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           throw new IOException("DIR* NameSystem.startFile: " +
           throw new IOException("DIR* NameSystem.startFile: " +
                                 "Unable to add file to namespace.");
                                 "Unable to add file to namespace.");
         }
         }
-        leaseManager.addLease(newNode.clientName, src);
+        leaseManager.addLease(newNode.getClientName(), src);
         if (NameNode.stateChangeLog.isDebugEnabled()) {
         if (NameNode.stateChangeLog.isDebugEnabled()) {
           NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
           NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
                                      +"add "+src+" to namespace for "+holder);
                                      +"add "+src+" to namespace for "+holder);
@@ -1048,40 +1091,36 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     LocatedBlock lb = null;
     LocatedBlock lb = null;
     synchronized (this) {
     synchronized (this) {
       INodeFileUnderConstruction file = (INodeFileUnderConstruction)dir.getFileINode(src);
       INodeFileUnderConstruction file = (INodeFileUnderConstruction)dir.getFileINode(src);
-
-      BlockInfo[] blocks = file.getBlocks();
-      if (blocks != null && blocks.length > 0) {
-        BlockInfo last = blocks[blocks.length-1];
-        // this is a redundant search in blocksMap
-        // should be resolved by the new implementation of append
-        BlockInfo storedBlock = blockManager.getStoredBlock(last);
-        assert last == storedBlock : "last block should be in the blocksMap";
-        if (file.getPreferredBlockSize() > storedBlock.getNumBytes()) {
+      BlockInfo lastBlock = file.getLastBlock();
+      if (lastBlock != null) {
+        assert lastBlock == blockManager.getStoredBlock(lastBlock) :
+          "last block of the file is not in blocksMap";
+        if (file.getPreferredBlockSize() > lastBlock.getNumBytes()) {
           long fileLength = file.computeContentSummary().getLength();
           long fileLength = file.computeContentSummary().getLength();
-          DatanodeDescriptor[] targets = blockManager.getNodes(storedBlock);
+          DatanodeDescriptor[] targets = blockManager.getNodes(lastBlock);
           // remove the replica locations of this block from the node
           // remove the replica locations of this block from the node
           for (int i = 0; i < targets.length; i++) {
           for (int i = 0; i < targets.length; i++) {
-            targets[i].removeBlock(storedBlock);
+            targets[i].removeBlock(lastBlock);
           }
           }
-          // set the locations of the last block in the lease record
-          file.setLastBlock(storedBlock, targets);
+          // convert last block to under-construction and set its locations
+          blockManager.convertLastBlockToUnderConstruction(file, targets);
 
 
-          lb = new LocatedBlock(last, targets, 
-                                fileLength-storedBlock.getNumBytes());
+          lb = new LocatedBlock(lastBlock, targets, 
+                                fileLength-lastBlock.getNumBytes());
           if (isAccessTokenEnabled) {
           if (isAccessTokenEnabled) {
             lb.setAccessToken(accessTokenHandler.generateToken(lb.getBlock()
             lb.setAccessToken(accessTokenHandler.generateToken(lb.getBlock()
                 .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
                 .getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
           }
           }
 
 
           // Remove block from replication queue.
           // Remove block from replication queue.
-          blockManager.updateNeededReplications(last, 0, 0);
+          blockManager.updateNeededReplications(lastBlock, 0, 0);
 
 
           // remove this block from the list of pending blocks to be deleted. 
           // remove this block from the list of pending blocks to be deleted. 
           // This reduces the possibility of triggering HADOOP-1349.
           // This reduces the possibility of triggering HADOOP-1349.
           //
           //
           for (DatanodeDescriptor dd : targets) {
           for (DatanodeDescriptor dd : targets) {
             String datanodeId = dd.getStorageID();
             String datanodeId = dd.getStorageID();
-            blockManager.removeFromInvalidates(datanodeId, last);
+            blockManager.removeFromInvalidates(datanodeId, lastBlock);
           }
           }
         }
         }
       }
       }
@@ -1115,7 +1154,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * client to "try again later".
    * client to "try again later".
    */
    */
   public LocatedBlock getAdditionalBlock(String src, 
   public LocatedBlock getAdditionalBlock(String src, 
-                                         String clientName
+                                         String clientName,
+                                         Block previous
                                          ) throws IOException {
                                          ) throws IOException {
     long fileLength, blockSize;
     long fileLength, blockSize;
     int replication;
     int replication;
@@ -1135,6 +1175,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
 
 
+      // commit the last block
+      blockManager.commitLastBlock(pendingFile, previous);
+
       //
       //
       // If we fail this, bad things happen!
       // If we fail this, bad things happen!
       //
       //
@@ -1168,9 +1211,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         throw new NotReplicatedYetException("Not replicated yet:" + src);
         throw new NotReplicatedYetException("Not replicated yet:" + src);
       }
       }
 
 
+      // complete the penultimate block
+      blockManager.completeBlock(pendingFile, pendingFile.numBlocks()-2);
+
       // allocate new block record block locations in INode.
       // allocate new block record block locations in INode.
-      newBlock = allocateBlock(src, pathINodes);
-      pendingFile.setTargets(targets);
+      newBlock = allocateBlock(src, pathINodes, targets);
       
       
       for (DatanodeDescriptor dn : targets) {
       for (DatanodeDescriptor dn : targets) {
         dn.incBlocksScheduled();
         dn.incBlocksScheduled();
@@ -1250,15 +1295,18 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     COMPLETE_SUCCESS
     COMPLETE_SUCCESS
   }
   }
   
   
-  public CompleteFileStatus completeFile(String src, String holder) throws IOException {
-    CompleteFileStatus status = completeFileInternal(src, holder);
+  public CompleteFileStatus completeFile(String src,
+                                         String holder,
+                                         Block last) throws IOException {
+    CompleteFileStatus status = completeFileInternal(src, holder, last);
     getEditLog().logSync();
     getEditLog().logSync();
     return status;
     return status;
   }
   }
 
 
-
-  private synchronized CompleteFileStatus completeFileInternal(String src, 
-                                                String holder) throws IOException {
+  private synchronized CompleteFileStatus completeFileInternal(
+                                            String src, 
+                                            String holder,
+                                            Block last) throws IOException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
     NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
     if (isInSafeMode())
     if (isInSafeMode())
       throw new SafeModeException("Cannot complete file " + src, safeMode);
       throw new SafeModeException("Cannot complete file " + src, safeMode);
@@ -1279,7 +1327,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                      ("from " + pendingFile.getClientMachine()))
                                      ("from " + pendingFile.getClientMachine()))
                                   );                      
                                   );                      
       return CompleteFileStatus.OPERATION_FAILED;
       return CompleteFileStatus.OPERATION_FAILED;
-    } else if (!checkFileProgress(pendingFile, true)) {
+    } 
+
+    // commit the last block
+    blockManager.commitLastBlock(pendingFile, last);
+
+    if (!checkFileProgress(pendingFile, true)) {
       return CompleteFileStatus.STILL_WAITING;
       return CompleteFileStatus.STILL_WAITING;
     }
     }
 
 
@@ -1312,13 +1365,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * @param inodes INode representing each of the components of src. 
    * @param inodes INode representing each of the components of src. 
    *        <code>inodes[inodes.length-1]</code> is the INode for the file.
    *        <code>inodes[inodes.length-1]</code> is the INode for the file.
    */
    */
-  private Block allocateBlock(String src, INode[] inodes) throws IOException {
+  private Block allocateBlock(String src,
+                              INode[] inodes,
+                              DatanodeDescriptor targets[]) throws IOException {
     Block b = new Block(FSNamesystem.randBlockId.nextLong(), 0, 0); 
     Block b = new Block(FSNamesystem.randBlockId.nextLong(), 0, 0); 
     while(isValidBlock(b)) {
     while(isValidBlock(b)) {
       b.setBlockId(FSNamesystem.randBlockId.nextLong());
       b.setBlockId(FSNamesystem.randBlockId.nextLong());
     }
     }
     b.setGenerationStamp(getGenerationStamp());
     b.setGenerationStamp(getGenerationStamp());
-    b = dir.addBlock(src, inodes, b);
+    b = dir.addBlock(src, inodes, b, targets);
     NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
     NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
                                  +src+ ". "+b);
                                  +src+ ". "+b);
     return b;
     return b;
@@ -1329,12 +1384,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * replicated.  If not, return false. If checkall is true, then check
    * replicated.  If not, return false. If checkall is true, then check
    * all blocks, otherwise check only penultimate block.
    * all blocks, otherwise check only penultimate block.
    */
    */
-  synchronized boolean checkFileProgress(INodeFile v, boolean checkall) {
+  synchronized boolean checkFileProgress(INodeFile v, boolean checkall) throws IOException {
     if (checkall) {
     if (checkall) {
       //
       //
       // check all blocks of the file.
       // check all blocks of the file.
       //
       //
-      for (Block block: v.getBlocks()) {
+      for (BlockInfo block: v.getBlocks()) {
         if (!blockManager.checkMinReplication(block)) {
         if (!blockManager.checkMinReplication(block)) {
           return false;
           return false;
         }
         }
@@ -1343,7 +1398,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       //
       //
       // check the penultimate block of this file
       // check the penultimate block of this file
       //
       //
-      Block b = v.getPenultimateBlock();
+      BlockInfo b = v.getPenultimateBlock();
       if (b != null && !blockManager.checkMinReplication(b)) {
       if (b != null && !blockManager.checkMinReplication(b)) {
         return false;
         return false;
       }
       }
@@ -1614,20 +1669,31 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * Move a file that is being written to be immutable.
    * Move a file that is being written to be immutable.
    * @param src The filename
    * @param src The filename
    * @param lease The lease for the client creating the file
    * @param lease The lease for the client creating the file
+   * @param recoveryLeaseHolder reassign lease to this holder if the last block
+   *        needs recovery; keep current holder if null.
+   * @throws AlreadyBeingCreatedException if file is waiting to achieve minimal
+   *         replication;<br>
+   *         RecoveryInProgressException if lease recovery is in progress.<br>
+   *         IOException in case of an error.
+   * @return true  if file has been successfully finalized and closed or 
+   *         false if block recovery has been initiated
    */
    */
-  void internalReleaseLease(Lease lease, String src) throws IOException {
+  boolean internalReleaseLease(
+      Lease lease, String src, String recoveryLeaseHolder)
+  throws AlreadyBeingCreatedException,
+         IOException {
     LOG.info("Recovering lease=" + lease + ", src=" + src);
     LOG.info("Recovering lease=" + lease + ", src=" + src);
 
 
     INodeFile iFile = dir.getFileINode(src);
     INodeFile iFile = dir.getFileINode(src);
     if (iFile == null) {
     if (iFile == null) {
-      final String message = "DIR* NameSystem.internalReleaseCreate: "
+      final String message = "DIR* NameSystem.internalReleaseLease: "
         + "attempt to release a create lock on "
         + "attempt to release a create lock on "
         + src + " file does not exist.";
         + src + " file does not exist.";
       NameNode.stateChangeLog.warn(message);
       NameNode.stateChangeLog.warn(message);
       throw new IOException(message);
       throw new IOException(message);
     }
     }
     if (!iFile.isUnderConstruction()) {
     if (!iFile.isUnderConstruction()) {
-      final String message = "DIR* NameSystem.internalReleaseCreate: "
+      final String message = "DIR* NameSystem.internalReleaseLease: "
         + "attempt to release a create lock on "
         + "attempt to release a create lock on "
         + src + " but file is already closed.";
         + src + " but file is already closed.";
       NameNode.stateChangeLog.warn(message);
       NameNode.stateChangeLog.warn(message);
@@ -1635,39 +1701,123 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
     }
 
 
     INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) iFile;
     INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction) iFile;
+    int nrBlocks = pendingFile.numBlocks();
+    BlockInfo[] blocks = pendingFile.getBlocks();
+
+    int nrCompleteBlocks;
+    BlockInfo curBlock = null;
+    for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
+      curBlock = blocks[nrCompleteBlocks];
+      if(!curBlock.isComplete())
+        break;
+      assert blockManager.checkMinReplication(curBlock) :
+              "A COMPLETE block is not minimally replicated in " + src;
+    }
+
+    // If there are no incomplete blocks associated with this file,
+    // then reap lease immediately and close the file.
+    if(nrCompleteBlocks == nrBlocks) {
+      finalizeINodeFileUnderConstruction(src, pendingFile);
+      NameNode.stateChangeLog.warn("BLOCK*"
+        + " internalReleaseLease: All existing blocks are COMPLETE,"
+        + " lease removed, file closed.");
+      return true;  // closed!
+    }
+
+    // Only the last and the penultimate blocks may be in non COMPLETE state.
+    // If the penultimate block is not COMPLETE, then it must be COMMITTED.
+    if(nrCompleteBlocks < nrBlocks - 2 ||
+       nrCompleteBlocks == nrBlocks - 2 &&
+         curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
+      final String message = "DIR* NameSystem.internalReleaseLease: "
+        + "attempt to release a create lock on "
+        + src + " but file is already closed.";
+      NameNode.stateChangeLog.warn(message);
+      throw new IOException(message);
+    }
 
 
-    // Initialize lease recovery for pendingFile. If there are no blocks 
-    // associated with this file, then reap lease immediately. Otherwise 
-    // renew the lease and trigger lease recovery.
-    if (pendingFile.getTargets() == null ||
-        pendingFile.getTargets().length == 0) {
-      if (pendingFile.getBlocks().length == 0) {
+    // no we know that the last block is not COMPLETE, and
+    // that the penultimate block if exists is either COMPLETE or COMMITTED
+    BlockInfoUnderConstruction lastBlock = pendingFile.getLastBlock();
+    BlockUCState lastBlockState = lastBlock.getBlockUCState();
+    BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
+    BlockUCState penultimateBlockState = (penultimateBlock == null ?
+        BlockUCState.COMPLETE : penultimateBlock.getBlockUCState());
+    assert penultimateBlockState == BlockUCState.COMPLETE ||
+           penultimateBlockState == BlockUCState.COMMITTED :
+           "Unexpected state of penultimate block in " + src;
+
+    switch(lastBlockState) {
+    case COMPLETE:
+      assert false : "Already checked that the last block is incomplete";
+      break;
+    case COMMITTED:
+      // Close file if committed blocks are minimally replicated
+      if(blockManager.checkMinReplication(penultimateBlock) &&
+          blockManager.checkMinReplication(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile);
         finalizeINodeFileUnderConstruction(src, pendingFile);
         NameNode.stateChangeLog.warn("BLOCK*"
         NameNode.stateChangeLog.warn("BLOCK*"
-          + " internalReleaseLease: No blocks found, lease removed.");
-        return;
+          + " internalReleaseLease: Committed blocks are minimally replicated,"
+          + " lease removed, file closed.");
+        return true;  // closed!
       }
       }
-      // setup the Inode.targets for the last block from the blockManager
-      //
-      BlockInfo[] blocks = pendingFile.getBlocks();
-      BlockInfo last = blocks[blocks.length-1];
-      DatanodeDescriptor[] targets = blockManager.getNodes(last);
-      pendingFile.setTargets(targets);
+      // Cannot close file right now, since some blocks 
+      // are not yet minimally replicated.
+      // This may potentially cause infinite loop in lease recovery
+      // if there are no valid replicas on data-nodes.
+      String message = "DIR* NameSystem.internalReleaseLease: " +
+          "Failed to release lease for file " + src +
+          ". Committed blocks are waiting to be minimally replicated." +
+          " Try again later.";
+      NameNode.stateChangeLog.warn(message);
+      throw new AlreadyBeingCreatedException(message);
+    case UNDER_CONSTRUCTION:
+    case UNDER_RECOVERY:
+      // setup the last block locations from the blockManager if not known
+      if(lastBlock.getNumExpectedLocations() == 0)
+        lastBlock.setExpectedLocations(blockManager.getNodes(lastBlock));
+      // start recovery of the last block for this file
+      long blockRecoveryId = nextGenerationStamp();
+      lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
+      lastBlock.initializeBlockRecovery(blockRecoveryId);
+      leaseManager.renewLease(lease);
+      // Cannot close file right now, since the last block requires recovery.
+      // This may potentially cause infinite loop in lease recovery
+      // if there are no valid replicas on data-nodes.
+      NameNode.stateChangeLog.warn(
+                "DIR* NameSystem.internalReleaseLease: " +
+                "File " + src + " has not been closed." +
+               " Lease recovery is in progress. " +
+                "RecoveryId = " + blockRecoveryId + " for block " + lastBlock);
+      break;
     }
     }
-    // start lease recovery of the last block for this file.
-    pendingFile.assignPrimaryDatanode();
-    leaseManager.renewLease(lease);
+    return false;
   }
   }
 
 
-  private void finalizeINodeFileUnderConstruction(String src,
+  Lease reassignLease(Lease lease, String src, String newHolder,
+                      INodeFileUnderConstruction pendingFile) {
+    if(newHolder == null)
+      return lease;
+    pendingFile.setClientName(newHolder);
+    return leaseManager.reassignLease(lease, src, newHolder);
+  }
+
+
+  private void finalizeINodeFileUnderConstruction(
+      String src,
       INodeFileUnderConstruction pendingFile) throws IOException {
       INodeFileUnderConstruction pendingFile) throws IOException {
-    leaseManager.removeLease(pendingFile.clientName, src);
+    leaseManager.removeLease(pendingFile.getClientName(), src);
+
+    // complete the penultimate block
+    blockManager.completeBlock(pendingFile, pendingFile.numBlocks()-2);
 
 
     // The file is no longer pending.
     // The file is no longer pending.
-    // Create permanent INode, update blockmap
+    // Create permanent INode, update blocks
     INodeFile newFile = pendingFile.convertToInodeFile();
     INodeFile newFile = pendingFile.convertToInodeFile();
     dir.replaceNode(src, pendingFile, newFile);
     dir.replaceNode(src, pendingFile, newFile);
 
 
+    // complete last block of the file
+    blockManager.completeBlock(newFile, newFile.numBlocks()-1);
     // close file and persist block allocations for this file
     // close file and persist block allocations for this file
     dir.closeFile(src, newFile);
     dir.closeFile(src, newFile);
 
 
@@ -1690,11 +1840,20 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       throw new IOException("Block (=" + lastblock + ") not found");
       throw new IOException("Block (=" + lastblock + ") not found");
     }
     }
     INodeFile iFile = oldblockinfo.getINode();
     INodeFile iFile = oldblockinfo.getINode();
-    if (!iFile.isUnderConstruction()) {
+    if (!iFile.isUnderConstruction() || oldblockinfo.isComplete()) {
       throw new IOException("Unexpected block (=" + lastblock
       throw new IOException("Unexpected block (=" + lastblock
           + ") since the file (=" + iFile.getLocalName()
           + ") since the file (=" + iFile.getLocalName()
           + ") is not under construction");
           + ") is not under construction");
     }
     }
+
+    long recoveryId =
+      ((BlockInfoUnderConstruction)oldblockinfo).getBlockRecoveryId();
+    if(recoveryId != newgenerationstamp) {
+      throw new IOException("The recovery id " + newgenerationstamp
+          + " does not match current recovery id "
+          + recoveryId + " for block " + lastblock); 
+    }
+        
     INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)iFile;
     INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)iFile;
 
 
 
 
@@ -1703,12 +1862,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     blockManager.removeBlockFromMap(oldblockinfo);
     blockManager.removeBlockFromMap(oldblockinfo);
 
 
     if (deleteblock) {
     if (deleteblock) {
-      pendingFile.removeBlock(lastblock);
+      pendingFile.removeLastBlock(lastblock);
     }
     }
     else {
     else {
       // update last block, construct newblockinfo and add it to the blocks map
       // update last block, construct newblockinfo and add it to the blocks map
       lastblock.set(lastblock.getBlockId(), newlength, newgenerationstamp);
       lastblock.set(lastblock.getBlockId(), newlength, newgenerationstamp);
-      final BlockInfo newblockinfo = blockManager.addINode(lastblock, pendingFile);
+      BlockInfoUnderConstruction newblockinfo = 
+        new BlockInfoUnderConstruction(
+            lastblock, pendingFile.getReplication());
+      blockManager.addINode(newblockinfo, pendingFile);
 
 
       // find the DatanodeDescriptor objects
       // find the DatanodeDescriptor objects
       // There should be no locations in the blockManager till now because the
       // There should be no locations in the blockManager till now because the
@@ -1727,11 +1889,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         for (int i = 0; i < descriptors.length; i++) {
         for (int i = 0; i < descriptors.length; i++) {
           descriptors[i].addBlock(newblockinfo);
           descriptors[i].addBlock(newblockinfo);
         }
         }
-        pendingFile.setLastBlock(newblockinfo, null);
-      } else {
-        // add locations into the INodeUnderConstruction
-        pendingFile.setLastBlock(newblockinfo, descriptors);
       }
       }
+      // add locations into the INodeUnderConstruction
+      pendingFile.setLastBlock(newblockinfo, descriptors);
     }
     }
 
 
     // If this commit does not want to close the file, persist
     // If this commit does not want to close the file, persist
@@ -1745,7 +1905,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       LOG.info("commitBlockSynchronization(" + lastblock + ") successful");
       LOG.info("commitBlockSynchronization(" + lastblock + ") successful");
       return;
       return;
     }
     }
-    
+
+    // commit the last block
+    blockManager.commitLastBlock(pendingFile, lastblock);
+
     //remove lease, close file
     //remove lease, close file
     finalizeINodeFileUnderConstruction(src, pendingFile);
     finalizeINodeFileUnderConstruction(src, pendingFile);
     getEditLog().logSync();
     getEditLog().logSync();
@@ -3342,7 +3505,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   void setBlockTotal() {
   void setBlockTotal() {
     if (safeMode == null)
     if (safeMode == null)
       return;
       return;
-    safeMode.setBlockTotal((int)getBlocksTotal());
+    safeMode.setBlockTotal((int)getCompleteBlocksTotal());
   }
   }
 
 
   /**
   /**
@@ -3352,6 +3515,33 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return blockManager.getTotalBlocks();
     return blockManager.getTotalBlocks();
   }
   }
 
 
+  /**
+   * Get the total number of COMPLETE blocks in the system.
+   * For safe mode only complete blocks are counted.
+   */
+  long getCompleteBlocksTotal() {
+    // Calculate number of blocks under construction
+    long numUCBlocks = 0;
+    for (Lease lease : leaseManager.getSortedLeases()) {
+      for(String path : lease.getPaths()) {
+        INode node = dir.getFileINode(path);
+        assert node != null : "Found a lease for nonexisting file.";
+        assert node.isUnderConstruction() :
+          "Found a lease for file that is not under construction.";
+        INodeFileUnderConstruction cons = (INodeFileUnderConstruction) node;
+        BlockInfo[] blocks = cons.getBlocks();
+        if(blocks == null)
+          continue;
+        for(BlockInfo b : blocks) {
+          if(!b.isComplete())
+            numUCBlocks++;
+        }
+      }
+    }
+    LOG.info("Number of blocks under construction: " + numUCBlocks);
+    return getBlocksTotal() - numUCBlocks;
+  }
+
   /**
   /**
    * Enter safe mode manually.
    * Enter safe mode manually.
    * @throws IOException
    * @throws IOException
@@ -3671,29 +3861,129 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     return gs;
     return gs;
   }
   }
 
 
+  private INodeFileUnderConstruction checkUCBlock(Block block, String clientName) 
+  throws IOException {
+    // check safe mode
+    if (isInSafeMode())
+      throw new SafeModeException("Cannot get a new generation stamp and an " +
+                                "access token for block " + block, safeMode);
+    
+    // check stored block state
+    BlockInfo storedBlock = blockManager.getStoredBlock(block);
+    if (storedBlock == null || 
+        storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
+        throw new IOException(block + 
+            " does not exist or is not under Construction" + storedBlock);
+    }
+    
+    // check file inode
+    INodeFile file = storedBlock.getINode();
+    if (file==null || !file.isUnderConstruction()) {
+      throw new IOException("The file " + storedBlock + 
+          " is belonged to does not exist or it is not under construction.");
+    }
+    
+    // check lease
+    INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)file;
+    if (clientName == null || !clientName.equals(pendingFile.getClientName())) {
+      throw new LeaseExpiredException("Lease mismatch: " + block + 
+          " is accessed by a non lease holder " + clientName); 
+    }
+
+    return pendingFile;
+  }
+  
   /**
   /**
-   * Verifies that the block is associated with a file that has a lease.
-   * Increments, logs and then returns the stamp
+   * Get a new generation stamp together with an access token for 
+   * a block under construction
+   * 
+   * This method is called for recovering a failed pipeline or setting up
+   * a pipeline to append to a block.
+   * 
+   * @param block a block
+   * @param clientName the name of a client
+   * @return a located block with a new generation stamp and an access token
+   * @throws IOException if any error occurs
    */
    */
-  synchronized long nextGenerationStampForBlock(Block block) throws IOException {
-    BlockInfo storedBlock = blockManager.getStoredBlock(block);
-    if (storedBlock == null) {
-      String msg = block + " is already commited, storedBlock == null.";
-      LOG.info(msg);
-      throw new IOException(msg);
+  synchronized LocatedBlock updateBlockForPipeline(Block block, 
+      String clientName) throws IOException {
+    // check vadility of parameters
+    checkUCBlock(block, clientName);
+    
+    // get a new generation stamp and an access token
+    block.setGenerationStamp(nextGenerationStamp());
+    LocatedBlock locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
+    if (isAccessTokenEnabled) {
+      locatedBlock.setAccessToken(accessTokenHandler.generateToken(
+          block.getBlockId(), EnumSet.of(AccessTokenHandler.AccessMode.WRITE)));
     }
     }
-    INodeFile fileINode = storedBlock.getINode();
-    if (!fileINode.isUnderConstruction()) {
-      String msg = block + " is already commited, !fileINode.isUnderConstruction().";
-      LOG.info(msg);
+    return locatedBlock;
+  }
+  
+  
+  /**
+   * Update a pipeline for a block under construction
+   * 
+   * @param clientName the name of the client
+   * @param oldblock and old block
+   * @param newBlock a new block with a new generation stamp and length
+   * @param newNodes datanodes in the pipeline
+   * @throws IOException if any error occurs
+   */
+  synchronized void updatePipeline(String clientName, Block oldBlock, 
+      Block newBlock, DatanodeID[] newNodes)
+      throws IOException {
+    LOG.info("updatePipeline(block=" + oldBlock
+        + ", newGenerationStamp=" + newBlock.getGenerationStamp()
+        + ", newLength=" + newBlock.getNumBytes()
+        + ", newNodes=" + Arrays.asList(newNodes)
+        + ", clientName=" + clientName
+        + ")");
+
+    // check the vadility of the block and lease holder name
+    final INodeFileUnderConstruction pendingFile = 
+      checkUCBlock(oldBlock, clientName);
+    final BlockInfo oldblockinfo = pendingFile.getLastBlock();
+
+    // check new GS & length: this is not expected
+    if (newBlock.getGenerationStamp() <= oldblockinfo.getGenerationStamp() ||
+        newBlock.getNumBytes() < oldblockinfo.getNumBytes()) {
+      String msg = "Update " + oldBlock + " (len = " + 
+      oldblockinfo.getNumBytes() + ") to an older state: " + newBlock + 
+      " (len = " + newBlock.getNumBytes() +")";
+      LOG.warn(msg);
       throw new IOException(msg);
       throw new IOException(msg);
     }
     }
-    if (!((INodeFileUnderConstruction)fileINode).setLastRecoveryTime(now())) {
-      String msg = block + " is beening recovered, ignoring this request.";
-      LOG.info(msg);
-      throw new IOException(msg);
+    
+    // Remove old block from blocks map. This always have to be done
+    // because the generation stamp of this block is changing.
+    blockManager.removeBlockFromMap(oldblockinfo);
+
+    // update last block, construct newblockinfo and add it to the blocks map
+    BlockInfoUnderConstruction newblockinfo = 
+      new BlockInfoUnderConstruction(
+          newBlock, pendingFile.getReplication());
+    blockManager.addINode(newblockinfo, pendingFile);
+
+    // find the DatanodeDescriptor objects
+    DatanodeDescriptor[] descriptors = null;
+    if (newNodes.length > 0) {
+      descriptors = new DatanodeDescriptor[newNodes.length];
+      for(int i = 0; i < newNodes.length; i++) {
+        descriptors[i] = getDatanode(newNodes[i]);
+      }
+    }
+    // add locations into the INodeUnderConstruction
+    pendingFile.setLastBlock(newblockinfo, descriptors);
+
+    // persist blocks only if append is supported
+    String src = leaseManager.findPath(pendingFile);
+    if (supportAppends) {
+      dir.persistBlocks(src, pendingFile);
+      getEditLog().logSync();
     }
     }
-    return nextGenerationStamp();
+    LOG.info("updatePipeline(" + oldBlock + ") successfully to " + newBlock);
+    return;
   }
   }
 
 
   // rename was successful. If any part of the renamed subtree had
   // rename was successful. If any part of the renamed subtree had

+ 0 - 8
src/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -25,8 +25,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 
 
 /**
 /**
  * We keep an in-memory representation of the file/block hierarchy.
  * We keep an in-memory representation of the file/block hierarchy.
@@ -423,10 +421,4 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
     }
     }
     return null;
     return null;
   }
   }
-  
-  
-  LocatedBlocks createLocatedBlocks(List<LocatedBlock> blocks) {
-    return new LocatedBlocks(computeContentSummary().getLength(), blocks,
-        isUnderConstruction());
-  }
 }
 }

+ 43 - 6
src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -122,16 +122,29 @@ class INodeFile extends INode {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   long[] computeContentSummary(long[] summary) {
   long[] computeContentSummary(long[] summary) {
-    long bytes = 0;
-    for(Block blk : blocks) {
-      bytes += blk.getNumBytes();
-    }
-    summary[0] += bytes;
+    summary[0] += computeFileSize(true);
     summary[1]++;
     summary[1]++;
     summary[3] += diskspaceConsumed();
     summary[3] += diskspaceConsumed();
     return summary;
     return summary;
   }
   }
 
 
+  /** Compute file size.
+   * May or may not include BlockInfoUnderConstruction.
+   */
+  long computeFileSize(boolean includesBlockInfoUnderConstruction) {
+    if (blocks == null || blocks.length == 0) {
+      return 0;
+    }
+    final int last = blocks.length - 1;
+    //check if the last block is BlockInfoUnderConstruction
+    long bytes = blocks[last] instanceof BlockInfoUnderConstruction
+                 && !includesBlockInfoUnderConstruction?
+                     0: blocks[last].getNumBytes();
+    for(int i = 0; i < last; i++) {
+      bytes += blocks[i].getNumBytes();
+    }
+    return bytes;
+  }
   
   
 
 
   @Override
   @Override
@@ -173,13 +186,14 @@ class INodeFile extends INode {
   /**
   /**
    * Return the penultimate allocated block for this file.
    * Return the penultimate allocated block for this file.
    */
    */
-  Block getPenultimateBlock() {
+  BlockInfo getPenultimateBlock() {
     if (blocks == null || blocks.length <= 1) {
     if (blocks == null || blocks.length <= 1) {
       return null;
       return null;
     }
     }
     return blocks[blocks.length - 2];
     return blocks[blocks.length - 2];
   }
   }
 
 
+  // SHV !!! this is not used anywhere - remove
   INodeFileUnderConstruction toINodeFileUnderConstruction(
   INodeFileUnderConstruction toINodeFileUnderConstruction(
       String clientName, String clientMachine, DatanodeDescriptor clientNode
       String clientName, String clientMachine, DatanodeDescriptor clientNode
       ) throws IOException {
       ) throws IOException {
@@ -191,4 +205,27 @@ class INodeFile extends INode {
         blocks, getPermissionStatus(),
         blocks, getPermissionStatus(),
         clientName, clientMachine, clientNode);
         clientName, clientMachine, clientNode);
   }
   }
+
+  /**
+   * Get the last block of the file.
+   * Make sure it has the right type.
+   */
+  <T extends BlockInfo> T getLastBlock() throws IOException {
+    if (blocks == null || blocks.length == 0)
+      return null;
+    T returnBlock = null;
+    try {
+      @SuppressWarnings("unchecked")  // ClassCastException is caught below
+      T tBlock = (T)blocks[blocks.length - 1];
+      returnBlock = tBlock;
+    } catch(ClassCastException cce) {
+      throw new IOException("Unexpected last block type: " 
+          + blocks[blocks.length - 1].getClass().getSimpleName());
+    }
+    return returnBlock;
+  }
+
+  int numBlocks() {
+    return blocks == null ? 0 : blocks.length;
+  }
 }
 }

+ 22 - 63
src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -21,16 +21,13 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 
 
 
 
 class INodeFileUnderConstruction extends INodeFile {
 class INodeFileUnderConstruction extends INodeFile {
-  final String clientName;         // lease holder
+  private  String clientName;         // lease holder
   private final String clientMachine;
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
-
-  private int primaryNodeIndex = -1; //the node working on lease recovery
-  private DatanodeDescriptor[] targets = null;   //locations for last block
-  private long lastRecoveryTime = 0;
   
   
   INodeFileUnderConstruction(PermissionStatus permissions,
   INodeFileUnderConstruction(PermissionStatus permissions,
                              short replication,
                              short replication,
@@ -67,6 +64,10 @@ class INodeFileUnderConstruction extends INodeFile {
     return clientName;
     return clientName;
   }
   }
 
 
+  void setClientName(String clientName) {
+    this.clientName = clientName;
+  }
+
   String getClientMachine() {
   String getClientMachine() {
     return clientMachine;
     return clientMachine;
   }
   }
@@ -83,15 +84,6 @@ class INodeFileUnderConstruction extends INodeFile {
     return true;
     return true;
   }
   }
 
 
-  DatanodeDescriptor[] getTargets() {
-    return targets;
-  }
-
-  void setTargets(DatanodeDescriptor[] targets) {
-    this.targets = targets;
-    this.primaryNodeIndex = -1;
-  }
-
   //
   //
   // converts a INodeFileUnderConstruction into a INodeFile
   // converts a INodeFileUnderConstruction into a INodeFile
   // use the modification time as the access time
   // use the modification time as the access time
@@ -108,10 +100,10 @@ class INodeFileUnderConstruction extends INodeFile {
   }
   }
 
 
   /**
   /**
-   * remove a block from the block list. This block should be
+   * Remove a block from the block list. This block should be
    * the last one on the list.
    * the last one on the list.
    */
    */
-  void removeBlock(Block oldblock) throws IOException {
+  void removeLastBlock(Block oldblock) throws IOException {
     if (blocks == null) {
     if (blocks == null) {
       throw new IOException("Trying to delete non-existant block " + oldblock);
       throw new IOException("Trying to delete non-existant block " + oldblock);
     }
     }
@@ -124,57 +116,24 @@ class INodeFileUnderConstruction extends INodeFile {
     BlockInfo[] newlist = new BlockInfo[size_1];
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     blocks = newlist;
     blocks = newlist;
-    
-    // Remove the block locations for the last block.
-    targets = null;
   }
   }
 
 
-  synchronized void setLastBlock(BlockInfo newblock, DatanodeDescriptor[] newtargets
-      ) throws IOException {
-    if (blocks == null) {
-      throw new IOException("Trying to update non-existant block (newblock="
-          + newblock + ")");
-    }
-    blocks[blocks.length - 1] = newblock;
-    setTargets(newtargets);
-    lastRecoveryTime = 0;
-  }
-
-  /**
-   * Initialize lease recovery for this object
-   */
-  void assignPrimaryDatanode() {
-    //assign the first alive datanode as the primary datanode
-
-    if (targets.length == 0) {
-      NameNode.stateChangeLog.warn("BLOCK*"
-        + " INodeFileUnderConstruction.initLeaseRecovery:"
-        + " No blocks found, lease removed.");
-    }
-
-    int previous = primaryNodeIndex;
-    //find an alive datanode beginning from previous
-    for(int i = 1; i <= targets.length; i++) {
-      int j = (previous + i)%targets.length;
-      if (targets[j].isAlive) {
-        DatanodeDescriptor primary = targets[primaryNodeIndex = j]; 
-        primary.addBlockToBeRecovered(blocks[blocks.length - 1], targets);
-        NameNode.stateChangeLog.info("BLOCK* " + blocks[blocks.length - 1]
-          + " recovery started, primary=" + primary);
-        return;
-      }
-    }
-  }
-  
   /**
   /**
-   * Update lastRecoveryTime if expired.
-   * @return true if lastRecoveryTimeis updated. 
+   * Convert the last block of the file to an under-construction block.
+   * Set its locations.
    */
    */
-  synchronized boolean setLastRecoveryTime(long now) {
-    boolean expired = now - lastRecoveryTime > NameNode.LEASE_RECOVER_PERIOD;
-    if (expired) {
-      lastRecoveryTime = now;
+  BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
+                                          DatanodeDescriptor[] targets)
+  throws IOException {
+    if (blocks == null || blocks.length == 0) {
+      throw new IOException("Trying to update non-existant block. " +
+      		"File is empty.");
     }
     }
-    return expired;
+    BlockInfoUnderConstruction ucBlock =
+      lastBlock.convertToBlockUnderConstruction(
+          BlockUCState.UNDER_CONSTRUCTION, targets);
+    ucBlock.setINode(this);
+    setBlock(numBlocks()-1, ucBlock);
+    return ucBlock;
   }
   }
 }
 }

+ 26 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -102,7 +102,7 @@ public class LeaseManager {
   /**
   /**
    * Adds (or re-adds) the lease for the specified file.
    * Adds (or re-adds) the lease for the specified file.
    */
    */
-  synchronized void addLease(String holder, String src) {
+  synchronized Lease addLease(String holder, String src) {
     Lease lease = getLease(holder);
     Lease lease = getLease(holder);
     if (lease == null) {
     if (lease == null) {
       lease = new Lease(holder);
       lease = new Lease(holder);
@@ -113,6 +113,7 @@ public class LeaseManager {
     }
     }
     sortedLeasesByPath.put(src, lease);
     sortedLeasesByPath.put(src, lease);
     lease.paths.add(src);
     lease.paths.add(src);
+    return lease;
   }
   }
 
 
   /**
   /**
@@ -142,12 +143,23 @@ public class LeaseManager {
     }
     }
   }
   }
 
 
+  /**
+   * Reassign lease for file src to the new holder.
+   */
+  synchronized Lease reassignLease(Lease lease, String src, String newHolder) {
+    assert newHolder != null : "new lease holder is null";
+    if (lease != null) {
+      removeLease(lease, src);
+    }
+    return addLease(newHolder, src);
+  }
+
   /**
   /**
    * Finds the pathname for the specified pendingFile
    * Finds the pathname for the specified pendingFile
    */
    */
   synchronized String findPath(INodeFileUnderConstruction pendingFile
   synchronized String findPath(INodeFileUnderConstruction pendingFile
       ) throws IOException {
       ) throws IOException {
-    Lease lease = getLease(pendingFile.clientName);
+    Lease lease = getLease(pendingFile.getClientName());
     if (lease != null) {
     if (lease != null) {
       String src = lease.findPath(pendingFile);
       String src = lease.findPath(pendingFile);
       if (src != null) {
       if (src != null) {
@@ -265,7 +277,11 @@ public class LeaseManager {
     Collection<String> getPaths() {
     Collection<String> getPaths() {
       return paths;
       return paths;
     }
     }
-    
+
+    String getHolder() {
+      return holder;
+    }
+
     void replacePath(String oldpath, String newpath) {
     void replacePath(String oldpath, String newpath) {
       paths.remove(oldpath);
       paths.remove(oldpath);
       paths.add(newpath);
       paths.add(newpath);
@@ -376,7 +392,13 @@ public class LeaseManager {
       oldest.getPaths().toArray(leasePaths);
       oldest.getPaths().toArray(leasePaths);
       for(String p : leasePaths) {
       for(String p : leasePaths) {
         try {
         try {
-          fsnamesystem.internalReleaseLease(oldest, p);
+          if(fsnamesystem.internalReleaseLease(oldest, p, "HDFS_NameNode")) {
+            LOG.info("Lease recovery for file " + p +
+                          " is complete. File closed.");
+            removing.add(p);
+          } else
+            LOG.info("Started block recovery for file " + p +
+                          " lease " + oldest);
         } catch (IOException e) {
         } catch (IOException e) {
           LOG.error("Cannot release the path "+p+" in the lease "+oldest, e);
           LOG.error("Cannot release the path "+p+" in the lease "+oldest, e);
           removing.add(p);
           removing.add(p);

+ 20 - 7
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -610,11 +610,12 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
 
 
   /**
   /**
    */
    */
-  public LocatedBlock addBlock(String src, 
-                               String clientName) throws IOException {
+  public LocatedBlock addBlock(String src, String clientName,
+                               Block previous) throws IOException {
     stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
     stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
                          +src+" for "+clientName);
                          +src+" for "+clientName);
-    LocatedBlock locatedBlock = namesystem.getAdditionalBlock(src, clientName);
+    LocatedBlock locatedBlock = 
+      namesystem.getAdditionalBlock(src, clientName, previous);
     if (locatedBlock != null)
     if (locatedBlock != null)
       myMetrics.numAddBlockOps.inc();
       myMetrics.numAddBlockOps.inc();
     return locatedBlock;
     return locatedBlock;
@@ -633,9 +634,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
-  public boolean complete(String src, String clientName) throws IOException {
+  public boolean complete(String src, String clientName,
+                          Block last) throws IOException {
     stateChangeLog.debug("*DIR* NameNode.complete: " + src + " for " + clientName);
     stateChangeLog.debug("*DIR* NameNode.complete: " + src + " for " + clientName);
-    CompleteFileStatus returnCode = namesystem.completeFile(src, clientName);
+    CompleteFileStatus returnCode =
+      namesystem.completeFile(src, clientName, last);
     if (returnCode == CompleteFileStatus.STILL_WAITING) {
     if (returnCode == CompleteFileStatus.STILL_WAITING) {
       return false;
       return false;
     } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
     } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
@@ -664,10 +667,20 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
-  public long nextGenerationStamp(Block block) throws IOException{
-    return namesystem.nextGenerationStampForBlock(block);
+  @Override
+  public LocatedBlock updateBlockForPipeline(Block block, String clientName)
+  throws IOException {
+    return namesystem.updateBlockForPipeline(block, clientName);
   }
   }
 
 
+
+  @Override
+  public void updatePipeline(String clientName, Block oldBlock,
+      Block newBlock, DatanodeID[] newNodes)
+      throws IOException {
+    namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
+  }
+  
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   public void commitBlockSynchronization(Block block,
   public void commitBlockSynchronization(Block block,
       long newgenerationstamp, long newlength,
       long newgenerationstamp, long newlength,

+ 161 - 0
src/java/org/apache/hadoop/hdfs/server/protocol/BlockRecoveryCommand.java

@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * BlockRecoveryCommand is an instruction to a data-node to recover
+ * the specified blocks.
+ *
+ * The data-node that receives this command treats itself as a primary
+ * data-node in the recover process.
+ *
+ * Block recovery is identified by a recoveryId, which is also the new
+ * generation stamp, which the block will have after the recovery succeeds.
+ */
+public class BlockRecoveryCommand extends DatanodeCommand {
+  Collection<RecoveringBlock> recoveringBlocks;
+
+  /**
+   * This is a block with locations from which it should be recovered
+   * and the new generation stamp, which the block will have after 
+   * successful recovery.
+   * 
+   * The new generation stamp of the block, also plays role of the recovery id.
+   */
+  public static class RecoveringBlock extends LocatedBlock {
+    private long newGenerationStamp;
+
+    /**
+     * Create empty RecoveringBlock.
+     */
+    public RecoveringBlock() {
+      super();
+      newGenerationStamp = -1L;
+    }
+
+    /**
+     * Create RecoveringBlock.
+     */
+    public RecoveringBlock(Block b, DatanodeInfo[] locs, long newGS) {
+      super(b, locs, -1, false); // startOffset is unknown
+      this.newGenerationStamp = newGS;
+    }
+
+    /**
+     * Return the new generation stamp of the block,
+     * which also plays role of the recovery id.
+     */
+    public long getNewGenerationStamp() {
+      return newGenerationStamp;
+    }
+
+    ///////////////////////////////////////////
+    // Writable
+    ///////////////////////////////////////////
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (RecoveringBlock.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new RecoveringBlock(); }
+         });
+    }
+
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      out.writeLong(newGenerationStamp);
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      newGenerationStamp = in.readLong();
+    }
+  }
+
+  /**
+   * Create empty BlockRecoveryCommand.
+   */
+  public BlockRecoveryCommand() {
+    this(0);
+  }
+
+  /**
+   * Create BlockRecoveryCommand with
+   * the specified capacity for recovering blocks.
+   */
+  public BlockRecoveryCommand(int capacity) {
+    super(DatanodeProtocol.DNA_RECOVERBLOCK);
+    recoveringBlocks = new ArrayList<RecoveringBlock>(capacity);
+  }
+
+  /**
+   * Return the list of recovering blocks.
+   */
+  public Collection<RecoveringBlock> getRecoveringBlocks() {
+    return recoveringBlocks;
+  }
+
+  /**
+   * Add recovering block to the command.
+   */
+  public void add(RecoveringBlock block) {
+    recoveringBlocks.add(block);
+  }
+
+  ///////////////////////////////////////////
+  // Writable
+  ///////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (BlockRecoveryCommand.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new BlockRecoveryCommand(); }
+       });
+  }
+
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    out.writeInt(recoveringBlocks.size());
+    for(RecoveringBlock block : recoveringBlocks) {
+      block.write(out);
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    int numBlocks = in.readInt();
+    recoveringBlocks = new ArrayList<RecoveringBlock>(numBlocks);
+    for(int i = 0; i < numBlocks; i++) {
+      RecoveringBlock b = new RecoveringBlock();
+      b.readFields(in);
+      add(b);
+    }
+  }
+}

+ 2 - 9
src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -35,10 +35,9 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  **********************************************************************/
  **********************************************************************/
 public interface DatanodeProtocol extends VersionedProtocol {
 public interface DatanodeProtocol extends VersionedProtocol {
   /**
   /**
-   * 20: SendHeartbeat may return KeyUpdateCommand
-   *     Register returns access keys inside DatanodeRegistration object
+   * 23: nextGenerationStamp() removed.
    */
    */
-  public static final long versionID = 20L;
+  public static final long versionID = 23L;
   
   
   // error code
   // error code
   final static int NOTIFY = 0;
   final static int NOTIFY = 0;
@@ -142,12 +141,6 @@ public interface DatanodeProtocol extends VersionedProtocol {
    */
    */
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
   
   
-  /**
-   * @return the next GenerationStamp to be associated with the specified
-   * block. 
-   */
-  public long nextGenerationStamp(Block block) throws IOException;
-
   /**
   /**
    * Commit block synchronization in lease recovery
    * Commit block synchronization in lease recovery
    */
    */

+ 23 - 3
src/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 
 
 /** An inter-datanode protocol for updating generation stamp
 /** An inter-datanode protocol for updating generation stamp
@@ -31,17 +32,36 @@ public interface InterDatanodeProtocol extends VersionedProtocol {
   public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
   public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
 
 
   /**
   /**
-   * 3: added a finalize parameter to updateBlock
+   * 4: initReplicaRecovery(), updateReplicaUnderRecovery() added.
    */
    */
-  public static final long versionID = 3L;
+  public static final long versionID = 4L;
 
 
   /** @return the BlockMetaDataInfo of a block;
   /** @return the BlockMetaDataInfo of a block;
    *  null if the block is not found 
    *  null if the block is not found 
    */
    */
+  @Deprecated
   BlockMetaDataInfo getBlockMetaDataInfo(Block block) throws IOException;
   BlockMetaDataInfo getBlockMetaDataInfo(Block block) throws IOException;
 
 
   /**
   /**
    * Update the block to the new generation stamp and length.  
    * Update the block to the new generation stamp and length.  
    */
    */
-  void updateBlock(Block oldblock, Block newblock, boolean finalize) throws IOException;
+  @Deprecated
+  void updateBlock(Block oldblock, Block newblock, boolean finalize)
+  throws IOException;
+
+  /**
+   * Initialize a replica recovery.
+   * 
+   * @return actual state of the replica on this data-node or 
+   * null if data-node does not have the replica.
+   */
+  ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
+  throws IOException;
+
+  /**
+   * Update replica with the new generation stamp and length.  
+   */
+  Block updateReplicaUnderRecovery(Block oldBlock,
+                                   long recoveryId,
+                                   long newLength) throws IOException;
 }
 }

+ 81 - 0
src/java/org/apache/hadoop/hdfs/server/protocol/ReplicaRecoveryInfo.java

@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.protocol;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * Replica recovery information.
+ */
+public class ReplicaRecoveryInfo extends Block {
+  private ReplicaState originalState;
+
+  public ReplicaRecoveryInfo() {
+  }
+
+  public ReplicaRecoveryInfo(Block r, ReplicaState rState) {
+    super(r);
+    originalState = rState;
+  }
+
+  public ReplicaState getOriginalReplicaState() {
+    return originalState;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  ///////////////////////////////////////////
+  // Writable
+  ///////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (ReplicaRecoveryInfo.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new ReplicaRecoveryInfo(); }
+       });
+  }
+
+ @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    originalState = ReplicaState.read(in); 
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    originalState.write(out);
+  }
+}

+ 1 - 1
src/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -96,7 +96,7 @@ import org.apache.hadoop.io.WritableUtils;
 class ImageLoaderCurrent implements ImageLoader {
 class ImageLoaderCurrent implements ImageLoader {
   protected final DateFormat dateFormat = 
   protected final DateFormat dateFormat = 
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
-  private static int [] versions = {-16, -17, -18, -19};
+  private static int [] versions = {-16, -17, -18, -19, -20};
   private int imageVersion = 0;
   private int imageVersion = 0;
 
 
   /* (non-Javadoc)
   /* (non-Javadoc)

+ 65 - 0
src/test/aop/org/apache/hadoop/fi/FiHFlushTestUtil.java

@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fi;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fi.FiTestUtil.ActionContainer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+
+/** Helper methods and actions for hflush() fault injection tests */
+public class FiHFlushTestUtil extends DataTransferTestUtil {
+
+  /** {@inheritDoc} */
+  public static PipelineTest initTest() {
+    return thepipelinetest = new HFlushTest();
+  }
+  
+  /** Disk error action for fault injection tests */
+  public static class DerrAction extends DataTransferTestUtil.DataNodeAction {
+    /**
+     * @param currentTest The name of the test
+     * @param index       The index of the datanode
+     */
+    public DerrAction(String currentTest, int index) {
+      super(currentTest, index);
+    }
+
+    /** {@inheritDoc} */
+    public void run(DatanodeID id) throws IOException {
+      final Pipeline p = getPipelineTest().getPipeline(id);
+      if (p == null) {
+        FiTestUtil.LOG.info("FI: couldn't find a pipeline for " + id);
+        return;
+      }
+      if (p.contains(index, id)) {
+        final String s = super.toString(id);
+        FiTestUtil.LOG.info(s);
+        throw new DiskErrorException(s);
+      }
+    }
+  }
+  
+  /** Class adds new type of action */
+  public static class HFlushTest extends DataTransferTest {
+    public final ActionContainer<DatanodeID> fiCallHFlush = 
+      new ActionContainer<DatanodeID>();
+  }
+}

+ 13 - 12
src/test/aop/org/apache/hadoop/hdfs/DFSClientAspects.aj

@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fi.DataTransferTestUtil;
 import org.apache.hadoop.fi.DataTransferTestUtil;
+import org.apache.hadoop.fi.DataTransferTestUtil.DataTransferTest;
 import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream.DataStreamer;
 import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream.DataStreamer;
 
 
 import org.junit.Assert;
 import org.junit.Assert;
@@ -35,7 +36,7 @@ public aspect DFSClientAspects {
 
 
   before(DataStreamer datastreamer) : callCreateBlockOutputStream(datastreamer) {
   before(DataStreamer datastreamer) : callCreateBlockOutputStream(datastreamer) {
     Assert.assertFalse(datastreamer.hasError);
     Assert.assertFalse(datastreamer.hasError);
-    Assert.assertEquals(0, datastreamer.errorIndex);
+    Assert.assertEquals(-1, datastreamer.errorIndex);
   }
   }
 
 
   pointcut pipelineInitNonAppend(DataStreamer datastreamer):
   pointcut pipelineInitNonAppend(DataStreamer datastreamer):
@@ -48,8 +49,9 @@ public aspect DFSClientAspects {
         + datastreamer.hasError + " errorIndex=" + datastreamer.errorIndex);
         + datastreamer.hasError + " errorIndex=" + datastreamer.errorIndex);
     try {
     try {
       if (datastreamer.hasError) {
       if (datastreamer.hasError) {
-        DataTransferTestUtil.getDataTransferTest().fiPipelineInitErrorNonAppend
-            .run(datastreamer.errorIndex);
+        DataTransferTest dtTest = DataTransferTestUtil.getDataTransferTest();
+        if (dtTest != null )
+          dtTest.fiPipelineInitErrorNonAppend.run(datastreamer.errorIndex);
       }
       }
     } catch (IOException e) {
     } catch (IOException e) {
       throw new RuntimeException(e);
       throw new RuntimeException(e);
@@ -66,19 +68,18 @@ public aspect DFSClientAspects {
         + " errorIndex=" + datastreamer.errorIndex);
         + " errorIndex=" + datastreamer.errorIndex);
   }
   }
 
 
-  pointcut pipelineErrorAfterInit(boolean onError, boolean isAppend,
-      DataStreamer datastreamer):
-    call(* processDatanodeError(boolean, boolean))
-    && args(onError, isAppend)
-    && target(datastreamer)
-    && if(onError && !isAppend);
+  pointcut pipelineErrorAfterInit(DataStreamer datastreamer):
+    call(* processDatanodeError())
+    && within (DFSClient.DFSOutputStream.DataStreamer)
+    && target(datastreamer);
 
 
-  before(DataStreamer datastreamer) : pipelineErrorAfterInit(boolean, boolean, datastreamer) {
+  before(DataStreamer datastreamer) : pipelineErrorAfterInit(datastreamer) {
     LOG.info("FI: before pipelineErrorAfterInit: errorIndex="
     LOG.info("FI: before pipelineErrorAfterInit: errorIndex="
         + datastreamer.errorIndex);
         + datastreamer.errorIndex);
     try {
     try {
-      DataTransferTestUtil.getDataTransferTest().fiPipelineErrorAfterInit
-          .run(datastreamer.errorIndex);
+      DataTransferTest dtTest = DataTransferTestUtil.getDataTransferTest();
+      if (dtTest != null )
+        dtTest.fiPipelineErrorAfterInit.run(datastreamer.errorIndex);
     } catch (IOException e) {
     } catch (IOException e) {
       throw new RuntimeException(e);
       throw new RuntimeException(e);
     }
     }

+ 57 - 0
src/test/aop/org/apache/hadoop/hdfs/HFlushAspects.aj

@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fi.DataTransferTestUtil;
+import org.apache.hadoop.fi.FiHFlushTestUtil.HFlushTest;
+import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+
+public aspect HFlushAspects {
+  public static final Log LOG = LogFactory.getLog(HFlushAspects.class);
+
+  pointcut hflushCall (DFSOutputStream outstream) :
+    execution(void DFSOutputStream.hflush(..))
+    && target (outstream); 
+  
+  /** This advise is suppose to initiate a call to the action (fiCallHFlush)
+   *  which will throw DiskErrorException if a pipeline has been created
+   *  and datanodes used are belong to that very pipeline
+   */
+  after (DFSOutputStream streamer) throws IOException : hflushCall(streamer) {
+    LOG.info("FI: hflush for any datanode");    
+    LOG.info("FI: hflush " + thisJoinPoint.getThis());
+    DatanodeInfo[] nodes = streamer.getPipeline();
+    if (nodes == null) {
+        LOG.info("No pipeline is built");
+        return;
+    }
+    if (DataTransferTestUtil.getPipelineTest() == null) {
+        LOG.info("No test has been initialized");    
+        return;
+    }
+    for (int i=0; i<nodes.length; i++) {
+        ((HFlushTest)DataTransferTestUtil.getPipelineTest()).fiCallHFlush.run(nodes[i]);
+    }
+  }
+}

+ 175 - 0
src/test/aop/org/apache/hadoop/hdfs/TestFiHFlush.java

@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fi.FiHFlushTestUtil;
+import org.apache.hadoop.fi.FiTestUtil;
+import org.apache.hadoop.fi.FiHFlushTestUtil.DerrAction;
+import org.apache.hadoop.fi.FiHFlushTestUtil.HFlushTest;
+import org.eclipse.jdt.core.dom.ThisExpression;
+
+import java.io.IOException;
+
+/** Class provides basic fault injection tests according to the test plan
+    of HDFS-265
+ */
+public class TestFiHFlush {
+  
+  /** Methods initializes a test and sets required actions to be used later by
+   * an injected advice
+   * @param conf mini cluster configuration
+   * @param methodName String representation of a test method invoking this 
+   * method
+   * @param block_size needed size of file's block
+   * @param a is an action to be set for the set
+   * @throws IOException in case of any errors
+   */
+  private static void runDiskErrorTest (final Configuration conf, 
+      final String methodName, final int block_size, DerrAction a)
+      throws IOException {
+    FiTestUtil.LOG.info("Running " + methodName + " ...");
+    final HFlushTest hft = (HFlushTest) FiHFlushTestUtil.initTest();
+    hft.fiCallHFlush.set(a);
+    TestHFlush.doTheJob(conf, methodName, block_size, (short)3);
+  }
+  
+  /** The tests calls 
+   * {@link #runDiskErrorTest(Configuration, String, int, DerrAction)}
+   * to make a number of writes within a block boundaries.
+   * Although hflush() is called the test shouldn't expect an IOException
+   * in this case because the invocation is happening after write() call 
+   * is complete when pipeline doesn't exist anymore.
+   * Thus, injected fault won't be triggered for 0th datanode
+   */
+  @Test
+  public void hFlushFi01_a() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    runDiskErrorTest(new Configuration(), methodName, 
+        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 0));
+  }
+
+  /** The tests calls 
+   * {@link #runDiskErrorTest(Configuration, String, int, DerrAction)}
+   * to make a number of writes across a block boundaries.
+   * hflush() is called after each write() during a pipeline life time.
+   * Thus, injected fault ought to be triggered for 0th datanode
+   */
+  @Test(expected = IOException.class)
+  public void hFlushFi01_b() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 512;
+    int customBlockSize = customPerChecksumSize * 3;
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+    runDiskErrorTest(conf, methodName, 
+        customBlockSize, new DerrAction(methodName, 0));
+  }
+  
+  /** Similar to {@link #hFlushFi01_b()} but writing happens
+   * across block and checksum's boundaries
+   */
+  @Test(expected = IOException.class)
+  public void hFlushFi01_c() throws IOException { 
+    final String methodName = FiTestUtil.getMethodName();
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 400;
+    int customBlockSize = customPerChecksumSize * 3;
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+    runDiskErrorTest(conf, methodName, 
+        customBlockSize, new DerrAction(methodName, 0));
+  }
+
+  /** Similar to {@link #hFlushFi01_a()} but for a pipeline's 1st datanode
+   */
+  @Test
+  public void hFlushFi02_a() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    runDiskErrorTest(new Configuration(), methodName,
+        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 1));
+  }
+
+  /** Similar to {@link #hFlushFi01_b()} but for a pipeline's 1st datanode
+   */
+@Test(expected = IOException.class)
+  public void hFlushFi02_b() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 512;
+    int customBlockSize = customPerChecksumSize * 3;
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+    runDiskErrorTest(conf, methodName,
+        customBlockSize, new DerrAction(methodName, 1));
+  }
+
+  /** Similar to {@link #hFlushFi01_c()} but for a pipeline's 1st datanode
+   */
+  @Test(expected = IOException.class)
+  public void hFlushFi02_c() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 400;
+    int customBlockSize = customPerChecksumSize * 3;
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+    runDiskErrorTest(conf, methodName,
+        customBlockSize, new DerrAction(methodName, 1));
+  }
+  
+  /** Similar to {@link #hFlushFi01_a()} but for a pipeline's 2nd datanode
+   */
+  @Test
+  public void hFlushFi03_a() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    runDiskErrorTest(new Configuration(), methodName,
+        AppendTestUtil.BLOCK_SIZE, new DerrAction(methodName, 2));
+  }
+  
+  /** Similar to {@link #hFlushFi01_b()} but for a pipeline's 2nd datanode
+   */
+  @Test(expected = IOException.class)
+  public void hFlushFi03_b() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 512;
+    int customBlockSize = customPerChecksumSize * 3;
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+    runDiskErrorTest(conf, methodName,
+        customBlockSize, new DerrAction(methodName, 2));
+  }
+
+  /** Similar to {@link #hFlushFi01_c()} but for a pipeline's 2nd datanode
+   */
+  @Test(expected = IOException.class)
+  public void hFlushFi03_c() throws IOException {
+    final String methodName = FiTestUtil.getMethodName();
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 400;
+    int customBlockSize = customPerChecksumSize * 3;
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+    runDiskErrorTest(conf, methodName,
+        customBlockSize, new DerrAction(methodName, 2));
+  }
+}

+ 1 - 1
src/test/aop/org/apache/hadoop/hdfs/protocol/ClientProtocolAspects.aj

@@ -27,7 +27,7 @@ public aspect ClientProtocolAspects {
   public static final Log LOG = LogFactory.getLog(ClientProtocolAspects.class);
   public static final Log LOG = LogFactory.getLog(ClientProtocolAspects.class);
 
 
   pointcut addBlock():
   pointcut addBlock():
-    call(LocatedBlock ClientProtocol.addBlock(String, String));
+    call(LocatedBlock ClientProtocol.addBlock(String, String,..));
 
 
   after() returning(LocatedBlock lb): addBlock() {
   after() returning(LocatedBlock lb): addBlock() {
     PipelineTest pipelineTest = DataTransferTestUtil.getPipelineTest();
     PipelineTest pipelineTest = DataTransferTestUtil.getPipelineTest();

+ 3 - 2
src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj

@@ -60,8 +60,9 @@ public aspect DataTransferProtocolAspects {
     final DataNode d = dataxceiver.getDataNode();
     final DataNode d = dataxceiver.getDataNode();
     LOG.info("FI: statusRead " + status + ", datanode="
     LOG.info("FI: statusRead " + status + ", datanode="
         + d.getDatanodeRegistration().getName());    
         + d.getDatanodeRegistration().getName());    
-    DataTransferTestUtil.getDataTransferTest().fiStatusRead.run(
-        d.getDatanodeRegistration());
+    DataTransferTest dtTest = DataTransferTestUtil.getDataTransferTest();
+    if (dtTest != null) 
+      dtTest.fiStatusRead.run(d.getDatanodeRegistration());
   }
   }
 
 
   pointcut receiverOpWriteBlock(DataXceiver dataxceiver):
   pointcut receiverOpWriteBlock(DataXceiver dataxceiver):

+ 4 - 6
src/test/aop/org/apache/hadoop/hdfs/server/datanode/FSDatasetAspects.aj

@@ -35,11 +35,9 @@ public aspect FSDatasetAspects {
   // the following will inject faults inside of the method in question 		
   // the following will inject faults inside of the method in question 		
     execution (* FSDataset.getBlockFile(..)) && !within(FSDatasetAspects +);
     execution (* FSDataset.getBlockFile(..)) && !within(FSDatasetAspects +);
 
 
-  // the following will inject faults before the actual call of the method
-  // call (* FSDataset.getBlockFile(..)) && !within(FSDatasetAspects +);
-
-  pointcut callCreateBlockWriteStream() : 
-    call (BlockWriteStreams FSDataset.createBlockWriteStreams(..)) 
+  pointcut callCreateBlockWriteStream(ReplicaInPipeline repl) : 
+    call (BlockWriteStreams createStreams())
+    && target (repl)
       && !within(FSDatasetAspects +);
       && !within(FSDatasetAspects +);
 
 
   // This aspect specifies the logic of our fault point.
   // This aspect specifies the logic of our fault point.
@@ -54,7 +52,7 @@ public aspect FSDatasetAspects {
     }
     }
   }
   }
 
 
-  before() throws DiskOutOfSpaceException : callCreateBlockWriteStream() {
+  before(ReplicaInPipeline repl) throws DiskOutOfSpaceException : callCreateBlockWriteStream(repl) {
     if (ProbabilityModel.injectCriteria(FSDataset.class.getSimpleName())) {
     if (ProbabilityModel.injectCriteria(FSDataset.class.getSimpleName())) {
       LOG.info("Before the injection point");
       LOG.info("Before the injection point");
       Thread.dumpStack();
       Thread.dumpStack();

+ 10 - 5
src/test/findbugsExcludeFile.xml

@@ -208,17 +208,22 @@
      </Match>
      </Match>
 
 
      <!--
      <!--
-       CreateBlockWriteStreams and getTmpInputStreams are pretty much like a stream constructor.
+       getTmpInputStreams is pretty much like a stream constructor.
        The newly created streams are not supposed to be closed in the constructor. So ignore
        The newly created streams are not supposed to be closed in the constructor. So ignore
        the OBL warning.
        the OBL warning.
      -->
      -->
      <Match>
      <Match>
        <Class name="org.apache.hadoop.hdfs.server.datanode.FSDataset" />
        <Class name="org.apache.hadoop.hdfs.server.datanode.FSDataset" />
-       <Or>
-         <Method name="createBlockWriteStreams" />
-         <Method name="getTmpInputStreams" />
-       </Or>
+       <Method name="getTmpInputStreams" />
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
        <Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
      </Match>
      </Match>
 
 
+     <!--
+      ResponseProccessor is thread that is designed to catch RuntimeException.
+     -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer$ResponseProcessor" />
+       <Method name="run" />
+       <Bug pattern="REC_CATCH_EXCEPTION" />
+     </Match>
  </FindBugsFilter>
  </FindBugsFilter>

+ 12 - 5
src/test/hdfs/org/apache/hadoop/hdfs/AppendTestUtil.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 
 
 /** Utilities for append-related tests */ 
 /** Utilities for append-related tests */ 
-class AppendTestUtil {
+public class AppendTestUtil {
   /** For specifying the random number generator seed,
   /** For specifying the random number generator seed,
    *  change the following value:
    *  change the following value:
    */
    */
@@ -84,8 +84,15 @@ class AppendTestUtil {
       LOG.info("ms=" + ms, e);
       LOG.info("ms=" + ms, e);
     }
     }
   }
   }
-
-  static FileSystem createHdfsWithDifferentUsername(Configuration conf
+  
+  /**
+   * Returns the reference to a new instance of FileSystem created 
+   * with different user name
+   * @param conf current Configuration
+   * @return FileSystem instance
+   * @throws IOException
+   */
+  public static FileSystem createHdfsWithDifferentUsername(Configuration conf
       ) throws IOException {
       ) throws IOException {
     Configuration conf2 = new Configuration(conf);
     Configuration conf2 = new Configuration(conf);
     String username = UserGroupInformation.getCurrentUGI().getUserName()+"_XXX";
     String username = UserGroupInformation.getCurrentUGI().getUserName()+"_XXX";
@@ -134,7 +141,7 @@ class AppendTestUtil {
    *  Make sure to call close() on the returned stream
    *  Make sure to call close() on the returned stream
    *  @throws IOException an exception might be thrown
    *  @throws IOException an exception might be thrown
    */
    */
-  static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
+  public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
       throws IOException {
       throws IOException {
     return fileSys.create(name, true,
     return fileSys.create(name, true,
         fileSys.getConf().getInt("io.file.buffer.size", 4096),
         fileSys.getConf().getInt("io.file.buffer.size", 4096),
@@ -146,7 +153,7 @@ class AppendTestUtil {
    *  the specified byte[] buffer's content
    *  the specified byte[] buffer's content
    *  @throws IOException an exception might be thrown
    *  @throws IOException an exception might be thrown
    */
    */
-  static void checkFullFile(FileSystem fs, Path name, int len,
+  public static void checkFullFile(FileSystem fs, Path name, int len,
                             final byte[] compareContent, String message) throws IOException {
                             final byte[] compareContent, String message) throws IOException {
     FSDataInputStream stm = fs.open(name);
     FSDataInputStream stm = fs.open(name);
     byte[] actual = new byte[len];
     byte[] actual = new byte[len];

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -110,7 +110,7 @@ public class DFSTestUtil {
   /** create nFiles with random names and directory hierarchies
   /** create nFiles with random names and directory hierarchies
    *  with random (but reproducible) data in them.
    *  with random (but reproducible) data in them.
    */
    */
-  void createFiles(FileSystem fs, String topdir,
+  public void createFiles(FileSystem fs, String topdir,
                    short replicationFactor) throws IOException {
                    short replicationFactor) throws IOException {
     files = new MyFile[nFiles];
     files = new MyFile[nFiles];
     
     

+ 2 - 1
src/test/hdfs/org/apache/hadoop/hdfs/DataNodeCluster.java

@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecureRandom;
 import java.security.SecureRandom;
+import java.util.Arrays;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -200,7 +201,7 @@ public class DataNodeCluster {
           }
           }
           for (int i = 1; i <= replication; ++i) { 
           for (int i = 1; i <= replication; ++i) { 
             // inject blocks for dn_i into dn_i and replica in dn_i's neighbors 
             // inject blocks for dn_i into dn_i and replica in dn_i's neighbors 
-            mc.injectBlocks((i_dn + i- 1)% numDataNodes, blocks);
+            mc.injectBlocks((i_dn + i- 1)% numDataNodes, Arrays.asList(blocks));
             System.out.println("Injecting blocks of dn " + i_dn  + " into dn" + 
             System.out.println("Injecting blocks of dn " + i_dn  + " into dn" + 
                 ((i_dn + i- 1)% numDataNodes));
                 ((i_dn + i- 1)% numDataNodes));
           }
           }

+ 10 - 7
src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
@@ -78,6 +79,8 @@ public class MiniDFSCluster {
   private File base_dir;
   private File base_dir;
   private File data_dir;
   private File data_dir;
   
   
+  public final static String FINALIZED_DIR_NAME = "/current/finalized/";
+  
   
   
   /**
   /**
    * This null constructor is used only when wishing to start a data node cluster
    * This null constructor is used only when wishing to start a data node cluster
@@ -635,7 +638,7 @@ public class MiniDFSCluster {
     if (i < 0 || i >= dataNodes.size())
     if (i < 0 || i >= dataNodes.size())
       return false;
       return false;
     for (int dn = i*2; dn < i*2+2; dn++) {
     for (int dn = i*2; dn < i*2+2; dn++) {
-      File blockFile = new File(dataDir, "data" + (dn+1) + "/current/" +
+      File blockFile = new File(dataDir, "data" + (dn+1) + FINALIZED_DIR_NAME +
                                 blockName);
                                 blockName);
       System.out.println("Corrupting for: " + blockFile);
       System.out.println("Corrupting for: " + blockFile);
       if (blockFile.exists()) {
       if (blockFile.exists()) {
@@ -853,7 +856,7 @@ public class MiniDFSCluster {
    * @param dataNodeIndex - data node whose block report is desired - the index is same as for getDataNodes()
    * @param dataNodeIndex - data node whose block report is desired - the index is same as for getDataNodes()
    * @return the block report for the specified data node
    * @return the block report for the specified data node
    */
    */
-  public Block[] getBlockReport(int dataNodeIndex) {
+  public Iterable<Block> getBlockReport(int dataNodeIndex) {
     if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) {
     if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) {
       throw new IndexOutOfBoundsException();
       throw new IndexOutOfBoundsException();
     }
     }
@@ -864,11 +867,11 @@ public class MiniDFSCluster {
   /**
   /**
    * 
    * 
    * @return block reports from all data nodes
    * @return block reports from all data nodes
-   *    Block[] is indexed in the same order as the list of datanodes returned by getDataNodes()
+   *    BlockListAsLongs is indexed in the same order as the list of datanodes returned by getDataNodes()
    */
    */
-  public Block[][] getAllBlockReports() {
+  public Iterable<Block>[] getAllBlockReports() {
     int numDataNodes = dataNodes.size();
     int numDataNodes = dataNodes.size();
-    Block[][] result = new Block[numDataNodes][];
+    Iterable<Block>[] result = new BlockListAsLongs[numDataNodes];
     for (int i = 0; i < numDataNodes; ++i) {
     for (int i = 0; i < numDataNodes; ++i) {
      result[i] = getBlockReport(i);
      result[i] = getBlockReport(i);
     }
     }
@@ -885,7 +888,7 @@ public class MiniDFSCluster {
    *             if any of blocks already exist in the data node
    *             if any of blocks already exist in the data node
    *   
    *   
    */
    */
-  public void injectBlocks(int dataNodeIndex, Block[] blocksToInject) throws IOException {
+  public void injectBlocks(int dataNodeIndex, Iterable<Block> blocksToInject) throws IOException {
     if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) {
     if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) {
       throw new IndexOutOfBoundsException();
       throw new IndexOutOfBoundsException();
     }
     }
@@ -907,7 +910,7 @@ public class MiniDFSCluster {
    *             if any of blocks already exist in the data nodes
    *             if any of blocks already exist in the data nodes
    *             Note the rest of the blocks are not injected.
    *             Note the rest of the blocks are not injected.
    */
    */
-  public void injectBlocks(Block[][] blocksToInject) throws IOException {
+  public void injectBlocks(Iterable<Block>[] blocksToInject) throws IOException {
     if (blocksToInject.length >  dataNodes.size()) {
     if (blocksToInject.length >  dataNodes.size()) {
       throw new IndexOutOfBoundsException();
       throw new IndexOutOfBoundsException();
     }
     }

+ 3 - 3
src/test/hdfs/org/apache/hadoop/hdfs/TestBlockMissingException.java

@@ -123,14 +123,14 @@ public class TestBlockMissingException extends TestCase {
    * The Data directories for a datanode
    * The Data directories for a datanode
    */
    */
   private File[] getDataNodeDirs(int i) throws IOException {
   private File[] getDataNodeDirs(int i) throws IOException {
-    File base_dir = new File(System.getProperty("test.build.data"), "dfs/");
+    String base_dir = MiniDFSCluster.getBaseDirectory();
     File data_dir = new File(base_dir, "data");
     File data_dir = new File(base_dir, "data");
     File dir1 = new File(data_dir, "data"+(2*i+1));
     File dir1 = new File(data_dir, "data"+(2*i+1));
     File dir2 = new File(data_dir, "data"+(2*i+2));
     File dir2 = new File(data_dir, "data"+(2*i+2));
     if (dir1.isDirectory() && dir2.isDirectory()) {
     if (dir1.isDirectory() && dir2.isDirectory()) {
       File[] dir = new File[2];
       File[] dir = new File[2];
-      dir[0] = new File(dir1, "current");
-      dir[1] = new File(dir2, "current"); 
+      dir[0] = new File(dir1, MiniDFSCluster.FINALIZED_DIR_NAME);
+      dir[1] = new File(dir2, MiniDFSCluster.FINALIZED_DIR_NAME); 
       return dir;
       return dir;
     }
     }
     return new File[0];
     return new File[0];

+ 12 - 11
src/test/hdfs/org/apache/hadoop/hdfs/TestBlockReport.java

@@ -103,15 +103,16 @@ public class TestBlockReport {
     // mock with newly created blocks
     // mock with newly created blocks
     // I can't use DFSTestUtil.getAllBlocks(fs.open(filePath)) because it
     // I can't use DFSTestUtil.getAllBlocks(fs.open(filePath)) because it
     // will keep the file open which will prevent the effect of the test
     // will keep the file open which will prevent the effect of the test
-    Block[] blocks = locatedToBlocks(cluster.getNameNode().getBlockLocations(
+    ArrayList<Block> blocks = 
+      locatedToBlocks(cluster.getNameNode().getBlockLocations(
         filePath.toString(), FILE_START,
         filePath.toString(), FILE_START,
         AppendTestUtil.FILE_SIZE).getLocatedBlocks(), null);
         AppendTestUtil.FILE_SIZE).getLocatedBlocks(), null);
 
 
-    LOG.info("Number of blocks allocated " + blocks.length);
-    int[] newLengths = new int[blocks.length];
+    LOG.info("Number of blocks allocated " + blocks.size());
+    int[] newLengths = new int[blocks.size()];
     int tempLen;
     int tempLen;
-    for (int i = 0; i < blocks.length; i++) {
-      Block b = blocks[i];
+    for (int i = 0; i < blocks.size(); i++) {
+      Block b = blocks.get(i);
       LOG.debug("Block " + b.getBlockName() + " before\t" + "Size " +
       LOG.debug("Block " + b.getBlockName() + " before\t" + "Size " +
           b.getNumBytes());
           b.getNumBytes());
       LOG.debug("Setting new length");
       LOG.debug("Setting new length");
@@ -123,7 +124,7 @@ public class TestBlockReport {
     }
     }
     cluster.getNameNode().blockReport(
     cluster.getNameNode().blockReport(
         cluster.listDataNodes()[DN_N0].dnRegistration,
         cluster.listDataNodes()[DN_N0].dnRegistration,
-        BlockListAsLongs.convertToArrayLongs(blocks));
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs());
 
 
     List<LocatedBlock> blocksAfterReport =
     List<LocatedBlock> blocksAfterReport =
         DFSTestUtil.getAllBlocks(fs.open(filePath));
         DFSTestUtil.getAllBlocks(fs.open(filePath));
@@ -175,7 +176,7 @@ public class TestBlockReport {
     for (Integer aRemovedIndex : removedIndex) {
     for (Integer aRemovedIndex : removedIndex) {
       blocks2Remove.add(lBlocks.get(aRemovedIndex).getBlock());
       blocks2Remove.add(lBlocks.get(aRemovedIndex).getBlock());
     }
     }
-    Block[] blocks = locatedToBlocks(lBlocks, removedIndex);
+    ArrayList<Block> blocks = locatedToBlocks(lBlocks, removedIndex);
 
 
     LOG.debug("Number of blocks allocated " + lBlocks.size());
     LOG.debug("Number of blocks allocated " + lBlocks.size());
 
 
@@ -196,7 +197,7 @@ public class TestBlockReport {
 
 
     cluster.getNameNode().blockReport(
     cluster.getNameNode().blockReport(
         cluster.listDataNodes()[DN_N0].dnRegistration,
         cluster.listDataNodes()[DN_N0].dnRegistration,
-        BlockListAsLongs.convertToArrayLongs(blocks));
+        new BlockListAsLongs(blocks, null).getBlockListAsLongs());
 
 
     cluster.getNamesystem().computeDatanodeWork();
     cluster.getNamesystem().computeDatanodeWork();
 
 
@@ -218,8 +219,8 @@ public class TestBlockReport {
         blocks2Remove.size(), cluster.getNamesystem().getUnderReplicatedBlocks());
         blocks2Remove.size(), cluster.getNamesystem().getUnderReplicatedBlocks());
   }
   }
 
 
-  private Block[] locatedToBlocks(final List<LocatedBlock> locatedBlks,
-                                  List<Integer> positionsToRemove) {
+  private ArrayList<Block> locatedToBlocks(final List<LocatedBlock> locatedBlks,
+                                           List<Integer> positionsToRemove) {
     int substructLen = 0;
     int substructLen = 0;
     if (positionsToRemove != null) { // Need to allocated smaller array
     if (positionsToRemove != null) { // Need to allocated smaller array
       substructLen = positionsToRemove.size();
       substructLen = positionsToRemove.size();
@@ -233,7 +234,7 @@ public class TestBlockReport {
       }
       }
       newList.add(locatedBlks.get(i).getBlock());
       newList.add(locatedBlks.get(i).getBlock());
     }
     }
-    return newList.toArray(ret);
+    return newList;
   }
   }
 
 
   private List<File> findAllFiles(File top, FilenameFilter mask) {
   private List<File> findAllFiles(File top, FilenameFilter mask) {

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java

@@ -49,7 +49,7 @@ public class TestBlocksScheduledCounter extends TestCase {
       out.write(i);
       out.write(i);
     }
     }
     // flush to make sure a block is allocated.
     // flush to make sure a block is allocated.
-    ((DFSOutputStream)(out.getWrappedStream())).sync();
+    ((DFSOutputStream)(out.getWrappedStream())).hflush();
     
     
     ArrayList<DatanodeDescriptor> dnList = new ArrayList<DatanodeDescriptor>();
     ArrayList<DatanodeDescriptor> dnList = new ArrayList<DatanodeDescriptor>();
     cluster.getNamesystem().DFSNodesStatus(dnList, dnList);
     cluster.getNamesystem().DFSNodesStatus(dnList, dnList);

+ 118 - 0
src/test/hdfs/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java

@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.IOUtils;
+
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * This tests pipeline recovery related client protocol works correct or not.
+ */
+public class TestClientProtocolForPipelineRecovery {
+  
+  @Test public void testGetNewStamp() throws IOException {
+    int numDataNodes = 1;
+    Configuration conf = new Configuration();
+    conf.setBoolean("dfs.support.append", true);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    try {
+      cluster.waitActive();
+      FileSystem fileSys = cluster.getFileSystem();
+      NameNode namenode = cluster.getNameNode();
+
+      /* Test writing to finalized replicas */
+      Path file = new Path("dataprotocol.dat");    
+      DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
+      // get the first blockid for the file
+      Block firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);
+
+      // test getNewStampAndToken on a finalized block
+      try {
+        namenode.updateBlockForPipeline(firstBlock, "");
+        Assert.fail("Can not get a new GS from a finalized block");
+      } catch (IOException e) {
+        Assert.assertTrue(e.getMessage().contains("is not under Construction"));
+      }
+      
+      // test getNewStampAndToken on a non-existent block
+      try {
+        long newBlockId = firstBlock.getBlockId() + 1;
+        Block newBlock = new Block(newBlockId, 0, 
+            firstBlock.getGenerationStamp());
+        namenode.updateBlockForPipeline(newBlock, "");
+        Assert.fail("Cannot get a new GS from a non-existent block");
+      } catch (IOException e) {
+        Assert.assertTrue(e.getMessage().contains("does not exist"));
+      }
+
+      
+      /* Test RBW replicas */
+      // change first block to a RBW
+      DFSOutputStream out = null;
+      try {
+        out = (DFSOutputStream)(fileSys.append(file).
+            getWrappedStream()); 
+        out.write(1);
+        out.hflush();
+        FSDataInputStream in = null;
+        try {
+          in = fileSys.open(file);
+          firstBlock = DFSTestUtil.getAllBlocks(in).get(0).getBlock();
+        } finally {
+          IOUtils.closeStream(in);
+        }
+
+        // test non-lease holder
+        DFSClient dfs = ((DistributedFileSystem)fileSys).dfs;
+        try {
+          namenode.updateBlockForPipeline(firstBlock, "test" + dfs.clientName);
+          Assert.fail("Cannot get a new GS for a non lease holder");
+        } catch (LeaseExpiredException e) {
+          Assert.assertTrue(e.getMessage().startsWith("Lease mismatch"));
+        }
+
+        // test null lease holder
+        try {
+          namenode.updateBlockForPipeline(firstBlock, null);
+          Assert.fail("Cannot get a new GS for a null lease holder");
+        } catch (LeaseExpiredException e) {
+          Assert.assertTrue(e.getMessage().startsWith("Lease mismatch"));
+        }
+
+        // test getNewStampAndToken on a rbw block
+        namenode.updateBlockForPipeline(firstBlock, dfs.clientName);
+      } finally {
+        IOUtils.closeStream(out);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}

+ 8 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -136,7 +136,7 @@ public class TestDFSClientRetries extends TestCase {
       return versionID;
       return versionID;
     }
     }
 
 
-    public LocatedBlock addBlock(String src, String clientName)
+    public LocatedBlock addBlock(String src, String clientName, Block previous)
     throws IOException
     throws IOException
     {
     {
       num_calls++;
       num_calls++;
@@ -169,7 +169,7 @@ public class TestDFSClientRetries extends TestCase {
 
 
     public void abandonBlock(Block b, String src, String holder) throws IOException {}
     public void abandonBlock(Block b, String src, String holder) throws IOException {}
 
 
-    public boolean complete(String src, String clientName) throws IOException { return false; }
+    public boolean complete(String src, String clientName, Block last) throws IOException { return false; }
 
 
     public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {}
     public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {}
 
 
@@ -215,6 +215,12 @@ public class TestDFSClientRetries extends TestCase {
 
 
     public void setTimes(String src, long mtime, long atime) throws IOException {}
     public void setTimes(String src, long mtime, long atime) throws IOException {}
 
 
+    @Override public LocatedBlock updateBlockForPipeline(Block block, 
+        String clientName) throws IOException { return null; }
+
+    @Override public void updatePipeline(String clientName, Block oldblock,
+        Block newBlock, DatanodeID[] newNodes)
+        throws IOException {}
   }
   }
   
   
   public void testNotYetReplicatedErrors() throws IOException
   public void testNotYetReplicatedErrors() throws IOException

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -1094,7 +1094,7 @@ public class TestDFSShell extends TestCase {
   static List<File> getBlockFiles(MiniDFSCluster cluster) throws IOException {
   static List<File> getBlockFiles(MiniDFSCluster cluster) throws IOException {
     List<File> files = new ArrayList<File>();
     List<File> files = new ArrayList<File>();
     List<DataNode> datanodes = cluster.getDataNodes();
     List<DataNode> datanodes = cluster.getDataNodes();
-    Block[][] blocks = cluster.getAllBlockReports();
+    Iterable<Block>[] blocks = cluster.getAllBlockReports();
     for(int i = 0; i < blocks.length; i++) {
     for(int i = 0; i < blocks.length; i++) {
       FSDataset ds = (FSDataset)datanodes.get(i).getFSDataset();
       FSDataset ds = (FSDataset)datanodes.get(i).getFSDataset();
       for(Block b : blocks[i]) {
       for(Block b : blocks[i]) {

+ 191 - 47
src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java

@@ -41,9 +41,12 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -51,6 +54,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
+import org.junit.Test;
 
 
 /**
 /**
  * This tests data transfer protocol handling in the Datanode. It sends
  * This tests data transfer protocol handling in the Datanode. It sends
@@ -94,6 +98,7 @@ public class TestDataTransferProtocol extends TestCase {
       
       
       DataInputStream in = new DataInputStream(sock.getInputStream());
       DataInputStream in = new DataInputStream(sock.getInputStream());
       out.write(sendBuf.toByteArray());
       out.write(sendBuf.toByteArray());
+      out.flush();
       try {
       try {
         in.readFully(retBuf);
         in.readFully(retBuf);
       } catch (EOFException eof) {
       } catch (EOFException eof) {
@@ -137,7 +142,175 @@ public class TestDataTransferProtocol extends TestCase {
     in.readFully(arr);
     in.readFully(arr);
   }
   }
   
   
-  public void testDataTransferProtocol() throws IOException {
+  private void writeZeroLengthPacket(Block block, String description)
+  throws IOException {
+    sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
+    sendOut.writeInt(512);         // checksum size
+    sendOut.writeInt(8);           // size of packet
+    sendOut.writeLong(block.getNumBytes());          // OffsetInBlock
+    sendOut.writeLong(100);        // sequencenumber
+    sendOut.writeBoolean(true);    // lastPacketInBlock
+
+    sendOut.writeInt(0);           // chunk length
+    sendOut.writeInt(0);           // zero checksum
+        
+    //ok finally write a block with 0 len
+    SUCCESS.write(recvOut);
+    Text.writeString(recvOut, ""); // first bad node
+    recvOut.writeLong(100);        // sequencenumber
+    SUCCESS.write(recvOut);
+    sendRecvData(description, false);
+  }
+  
+  private void testWrite(Block block, BlockConstructionStage stage, long newGS,
+      String description, Boolean eofExcepted) throws IOException {
+    sendBuf.reset();
+    recvBuf.reset();
+    DataTransferProtocol.Sender.opWriteBlock(sendOut, 
+        block.getBlockId(), block.getGenerationStamp(), 0,
+        stage, newGS, block.getNumBytes(), block.getNumBytes(), "cl", null,
+        new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
+    if (eofExcepted) {
+      ERROR.write(recvOut);
+      sendRecvData(description, true);
+    } else if (stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+      //ok finally write a block with 0 len
+      SUCCESS.write(recvOut);
+      Text.writeString(recvOut, ""); // first bad node
+      sendRecvData(description, false);
+    } else {
+      writeZeroLengthPacket(block, description);
+    }
+  }
+  
+  @Test public void testOpWrite() throws IOException {
+    int numDataNodes = 1;
+    Configuration conf = new Configuration();
+    conf.setBoolean("dfs.support.append", true);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    try {
+      cluster.waitActive();
+      datanode = cluster.getDataNodes().get(0).dnRegistration;
+      dnAddr = NetUtils.createSocketAddr(datanode.getName());
+      FileSystem fileSys = cluster.getFileSystem();
+
+      /* Test writing to finalized replicas */
+      Path file = new Path("dataprotocol.dat");    
+      DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
+      // get the first blockid for the file
+      Block firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);
+      // test PIPELINE_SETUP_CREATE on a finalized block
+      testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L,
+          "Cannot create an existing block", true);
+      // test PIPELINE_DATA_STREAMING on a finalized block
+      testWrite(firstBlock, BlockConstructionStage.DATA_STREAMING, 0L,
+          "Unexpected stage", true);
+      // test PIPELINE_SETUP_STREAMING_RECOVERY on an existing block
+      long newGS = firstBlock.getGenerationStamp() + 1;
+      testWrite(firstBlock, 
+          BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY, 
+          newGS, "Cannot recover data streaming to a finalized replica", true);
+      // test PIPELINE_SETUP_APPEND on an existing block
+      newGS = firstBlock.getGenerationStamp() + 1;
+      testWrite(firstBlock, 
+          BlockConstructionStage.PIPELINE_SETUP_APPEND,
+          newGS, "Append to a finalized replica", false);
+      firstBlock.setGenerationStamp(newGS);
+      // test PIPELINE_SETUP_APPEND_RECOVERY on an existing block
+      file = new Path("dataprotocol1.dat");    
+      DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
+      firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);
+      newGS = firstBlock.getGenerationStamp() + 1;
+      testWrite(firstBlock, 
+          BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY, newGS,
+          "Recover appending to a finalized replica", false);
+      // test PIPELINE_CLOSE_RECOVERY on an existing block
+      file = new Path("dataprotocol2.dat");    
+      DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
+      firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);
+      newGS = firstBlock.getGenerationStamp() + 1;
+      testWrite(firstBlock, 
+          BlockConstructionStage.PIPELINE_CLOSE_RECOVERY, newGS,
+          "Recover failed close to a finalized replica", false);
+      firstBlock.setGenerationStamp(newGS);
+
+      /* Test writing to a new block */
+      long newBlockId = firstBlock.getBlockId() + 1;
+      Block newBlock = new Block(newBlockId, 0, 
+          firstBlock.getGenerationStamp());
+
+      // test PIPELINE_SETUP_CREATE on a new block
+      testWrite(newBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L,
+          "Create a new block", false);
+      // test PIPELINE_SETUP_STREAMING_RECOVERY on a new block
+      newGS = newBlock.getGenerationStamp() + 1;
+      newBlock.setBlockId(newBlock.getBlockId()+1);
+      testWrite(newBlock, 
+          BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY, newGS,
+          "Recover a new block", true);
+      
+      // test PIPELINE_SETUP_APPEND on a new block
+      newGS = newBlock.getGenerationStamp() + 1;
+      testWrite(newBlock, 
+          BlockConstructionStage.PIPELINE_SETUP_APPEND, newGS,
+          "Cannot append to a new block", true);
+
+      // test PIPELINE_SETUP_APPEND_RECOVERY on a new block
+      newBlock.setBlockId(newBlock.getBlockId()+1);
+      newGS = newBlock.getGenerationStamp() + 1;
+      testWrite(newBlock, 
+          BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY, newGS,
+          "Cannot append to a new block", true);
+
+      /* Test writing to RBW replicas */
+      Path file1 = new Path("dataprotocol1.dat");    
+      DFSTestUtil.createFile(fileSys, file1, 1L, (short)numDataNodes, 0L);
+      DFSOutputStream out = (DFSOutputStream)(fileSys.append(file1).
+          getWrappedStream()); 
+      out.write(1);
+      out.hflush();
+      FSDataInputStream in = fileSys.open(file1);
+      firstBlock = DFSTestUtil.getAllBlocks(in).get(0).getBlock();
+      firstBlock.setNumBytes(2L);
+      
+      try {
+        // test PIPELINE_SETUP_CREATE on a RBW block
+        testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L,
+            "Cannot create a RBW block", true);
+        // test PIPELINE_SETUP_APPEND on an existing block
+        newGS = newBlock.getGenerationStamp() + 1;
+        testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_APPEND,
+            newGS, "Cannot append to a RBW replica", true);
+        // test PIPELINE_SETUP_APPEND on an existing block
+        testWrite(firstBlock, 
+            BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY,
+            newGS, "Recover append to a RBW replica", false);
+        firstBlock.setGenerationStamp(newGS);
+        // test PIPELINE_SETUP_STREAMING_RECOVERY on a RBW block
+        file = new Path("dataprotocol2.dat");    
+        DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
+        out = (DFSOutputStream)(fileSys.append(file).
+            getWrappedStream()); 
+        out.write(1);
+        out.hflush();
+        in = fileSys.open(file);
+        firstBlock = DFSTestUtil.getAllBlocks(in).get(0).getBlock();
+        firstBlock.setNumBytes(2L);
+        newGS = firstBlock.getGenerationStamp() + 1;
+        testWrite(firstBlock, 
+            BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY,
+            newGS, "Recover a RBW replica", false);
+      } finally {
+        IOUtils.closeStream(in);
+        IOUtils.closeStream(out);
+      }
+
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+@Test  public void testDataTransferProtocol() throws IOException {
     Random random = new Random();
     Random random = new Random();
     int oneMil = 1024*1024;
     int oneMil = 1024*1024;
     Path file = new Path("dataprotocol.dat");
     Path file = new Path("dataprotocol.dat");
@@ -146,6 +319,7 @@ public class TestDataTransferProtocol extends TestCase {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setInt("dfs.replication", numDataNodes); 
     conf.setInt("dfs.replication", numDataNodes); 
     MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
+    try {
     cluster.waitActive();
     cluster.waitActive();
     DFSClient dfsClient = new DFSClient(
     DFSClient dfsClient = new DFSClient(
                  new InetSocketAddress("localhost", cluster.getNameNodePort()),
                  new InetSocketAddress("localhost", cluster.getNameNodePort()),
@@ -178,16 +352,10 @@ public class TestDataTransferProtocol extends TestCase {
     
     
     /* Test OP_WRITE_BLOCK */
     /* Test OP_WRITE_BLOCK */
     sendBuf.reset();
     sendBuf.reset();
-    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    WRITE_BLOCK.write(sendOut);
-    sendOut.writeLong(newBlockId); // block id
-    sendOut.writeLong(0);          // generation stamp
-    sendOut.writeInt(0);           // targets in pipeline 
-    sendOut.writeBoolean(false);   // recoveryFlag
-    Text.writeString(sendOut, "cl");// clientID
-    sendOut.writeBoolean(false); // no src node info
-    sendOut.writeInt(0);           // number of downstream targets
-    AccessToken.DUMMY_TOKEN.write(sendOut);
+    DataTransferProtocol.Sender.opWriteBlock(sendOut, 
+        newBlockId, 0L, 0,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
+        new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     
     
     // bad bytes per checksum
     // bad bytes per checksum
@@ -198,32 +366,10 @@ public class TestDataTransferProtocol extends TestCase {
 
 
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
-    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    WRITE_BLOCK.write(sendOut);
-    sendOut.writeLong(newBlockId);
-    sendOut.writeLong(0);          // generation stamp
-    sendOut.writeInt(0);           // targets in pipeline 
-    sendOut.writeBoolean(false);   // recoveryFlag
-    Text.writeString(sendOut, "cl");// clientID
-    sendOut.writeBoolean(false); // no src node info
-
-    // bad number of targets
-    sendOut.writeInt(-1-random.nextInt(oneMil));
-    ERROR.write(recvOut);
-    sendRecvData("bad targets len while writing block " + newBlockId, true);
-
-    sendBuf.reset();
-    recvBuf.reset();
-    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    WRITE_BLOCK.write(sendOut);
-    sendOut.writeLong(++newBlockId);
-    sendOut.writeLong(0);          // generation stamp
-    sendOut.writeInt(0);           // targets in pipeline 
-    sendOut.writeBoolean(false);   // recoveryFlag
-    Text.writeString(sendOut, "cl");// clientID
-    sendOut.writeBoolean(false); // no src node info
-    sendOut.writeInt(0);
-    AccessToken.DUMMY_TOKEN.write(sendOut);
+    DataTransferProtocol.Sender.opWriteBlock(sendOut,
+        ++newBlockId, 0L, 0,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
+        new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeInt(512);
     sendOut.writeInt(512);
     sendOut.writeInt(4);           // size of packet
     sendOut.writeInt(4);           // size of packet
@@ -243,16 +389,10 @@ public class TestDataTransferProtocol extends TestCase {
     // test for writing a valid zero size block
     // test for writing a valid zero size block
     sendBuf.reset();
     sendBuf.reset();
     recvBuf.reset();
     recvBuf.reset();
-    sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
-    WRITE_BLOCK.write(sendOut);
-    sendOut.writeLong(++newBlockId);
-    sendOut.writeLong(0);          // generation stamp
-    sendOut.writeInt(0);           // targets in pipeline 
-    sendOut.writeBoolean(false);   // recoveryFlag
-    Text.writeString(sendOut, "cl");// clientID
-    sendOut.writeBoolean(false); // no src node info
-    sendOut.writeInt(0);
-    AccessToken.DUMMY_TOKEN.write(sendOut);
+    DataTransferProtocol.Sender.opWriteBlock(sendOut, 
+        ++newBlockId, 0L, 0,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
+        new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeInt(512);         // checksum size
     sendOut.writeInt(512);         // checksum size
     sendOut.writeInt(8);           // size of packet
     sendOut.writeInt(8);           // size of packet
@@ -262,6 +402,7 @@ public class TestDataTransferProtocol extends TestCase {
 
 
     sendOut.writeInt(0);           // chunk length
     sendOut.writeInt(0);           // chunk length
     sendOut.writeInt(0);           // zero checksum
     sendOut.writeInt(0);           // zero checksum
+    sendOut.flush();
     //ok finally write a block with 0 len
     //ok finally write a block with 0 len
     SUCCESS.write(recvOut);
     SUCCESS.write(recvOut);
     Text.writeString(recvOut, ""); // first bad node
     Text.writeString(recvOut, ""); // first bad node
@@ -353,5 +494,8 @@ public class TestDataTransferProtocol extends TestCase {
     Text.writeString(sendOut, "cl");
     Text.writeString(sendOut, "cl");
     AccessToken.DUMMY_TOKEN.write(sendOut);
     AccessToken.DUMMY_TOKEN.write(sendOut);
     readFile(fileSys, file, fileLen);
     readFile(fileSys, file, fileLen);
+    } finally {
+      cluster.shutdown();
+    }
   }
   }
 }
 }

+ 10 - 10
src/test/hdfs/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java

@@ -134,8 +134,8 @@ public class TestDatanodeBlockScanner extends TestCase {
     File baseDir = new File(MiniDFSCluster.getBaseDirectory(), "data");
     File baseDir = new File(MiniDFSCluster.getBaseDirectory(), "data");
     boolean corrupted = false;
     boolean corrupted = false;
     for (int i=replica*2; i<replica*2+2; i++) {
     for (int i=replica*2; i<replica*2+2; i++) {
-      File blockFile = new File(baseDir, "data" + (i+1)+ "/current/" + 
-                               blockName);
+      File blockFile = new File(baseDir, "data" + (i+1) + 
+          MiniDFSCluster.FINALIZED_DIR_NAME + blockName);
       if (blockFile.exists()) {
       if (blockFile.exists()) {
         // Corrupt replica by writing random bytes into replica
         // Corrupt replica by writing random bytes into replica
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
@@ -175,7 +175,7 @@ public class TestDatanodeBlockScanner extends TestCase {
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       blockCount = blocks.get(0).getLocations().length;
       blockCount = blocks.get(0).getLocations().length;
       try {
       try {
-        LOG.info("Looping until expected blockCount of 3 is received");
+        LOG.info("Looping until expected blockCount of 3 is received: " + blockCount);
         Thread.sleep(1000);
         Thread.sleep(1000);
       } catch (InterruptedException ignore) {
       } catch (InterruptedException ignore) {
       }
       }
@@ -194,7 +194,7 @@ public class TestDatanodeBlockScanner extends TestCase {
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
                    getBlockLocations(file1.toString(), 0, Long.MAX_VALUE);
       blockCount = blocks.get(0).getLocations().length;
       blockCount = blocks.get(0).getLocations().length;
       try {
       try {
-        LOG.info("Looping until expected blockCount of 2 is received");
+        LOG.info("Looping until expected blockCount of 2 is received: " + blockCount);
         Thread.sleep(1000);
         Thread.sleep(1000);
       } catch (InterruptedException ignore) {
       } catch (InterruptedException ignore) {
       }
       }
@@ -412,8 +412,8 @@ public class TestDatanodeBlockScanner extends TestCase {
   static boolean changeReplicaLength(String blockName, int dnIndex, int lenDelta) throws IOException {
   static boolean changeReplicaLength(String blockName, int dnIndex, int lenDelta) throws IOException {
     File baseDir = new File(MiniDFSCluster.getBaseDirectory(), "data");
     File baseDir = new File(MiniDFSCluster.getBaseDirectory(), "data");
     for (int i=dnIndex*2; i<dnIndex*2+2; i++) {
     for (int i=dnIndex*2; i<dnIndex*2+2; i++) {
-      File blockFile = new File(baseDir, "data" + (i+1)+ "/current/" + 
-                               blockName);
+      File blockFile = new File(baseDir, "data" + (i+1) + 
+          MiniDFSCluster.FINALIZED_DIR_NAME + blockName);
       if (blockFile.exists()) {
       if (blockFile.exists()) {
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
         raFile.setLength(raFile.length()+lenDelta);
         raFile.setLength(raFile.length()+lenDelta);
@@ -427,10 +427,10 @@ public class TestDatanodeBlockScanner extends TestCase {
   private static void waitForBlockDeleted(String blockName, int dnIndex) 
   private static void waitForBlockDeleted(String blockName, int dnIndex) 
   throws IOException, InterruptedException {
   throws IOException, InterruptedException {
     File baseDir = new File(MiniDFSCluster.getBaseDirectory(), "data");
     File baseDir = new File(MiniDFSCluster.getBaseDirectory(), "data");
-    File blockFile1 = new File(baseDir, "data" + (2*dnIndex+1)+ "/current/" + 
-        blockName);
-    File blockFile2 = new File(baseDir, "data" + (2*dnIndex+2)+ "/current/" + 
-        blockName);
+    File blockFile1 = new File(baseDir, "data" + (2*dnIndex+1) + 
+        MiniDFSCluster.FINALIZED_DIR_NAME + blockName);
+    File blockFile2 = new File(baseDir, "data" + (2*dnIndex+2) + 
+        MiniDFSCluster.FINALIZED_DIR_NAME + blockName);
     while (blockFile1.exists() || blockFile2.exists()) {
     while (blockFile1.exists() || blockFile2.exists()) {
       Thread.sleep(100);
       Thread.sleep(100);
     }
     }

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -148,7 +148,7 @@ public class TestFileAppend extends TestCase {
         Block b = blocks.get(i).getBlock();
         Block b = blocks.get(i).getBlock();
         System.out.println("testCopyOnWrite detaching block " + b);
         System.out.println("testCopyOnWrite detaching block " + b);
         assertTrue("Detaching block " + b + " should have returned true",
         assertTrue("Detaching block " + b + " should have returned true",
-            dataset.detachBlock(b, 1));
+            dataset.unlinkBlock(b, 1));
       }
       }
 
 
       // Since the blocks were already detached earlier, these calls should
       // Since the blocks were already detached earlier, these calls should
@@ -158,7 +158,7 @@ public class TestFileAppend extends TestCase {
         Block b = blocks.get(i).getBlock();
         Block b = blocks.get(i).getBlock();
         System.out.println("testCopyOnWrite detaching block " + b);
         System.out.println("testCopyOnWrite detaching block " + b);
         assertTrue("Detaching block " + b + " should have returned false",
         assertTrue("Detaching block " + b + " should have returned false",
-            !dataset.detachBlock(b, 1));
+            !dataset.unlinkBlock(b, 1));
       }
       }
 
 
     } finally {
     } finally {

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset;
-import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
 
 
 /** This class implements some of tests posted in HADOOP-2658. */
 /** This class implements some of tests posted in HADOOP-2658. */
 public class TestFileAppend3 extends junit.framework.TestCase {
 public class TestFileAppend3 extends junit.framework.TestCase {
@@ -220,6 +219,7 @@ public class TestFileAppend3 extends junit.framework.TestCase {
     FSDataOutputStream out = fs.append(p);
     FSDataOutputStream out = fs.append(p);
     final int len2 = (int)BLOCK_SIZE/2; 
     final int len2 = (int)BLOCK_SIZE/2; 
     AppendTestUtil.write(out, len1, len2);
     AppendTestUtil.write(out, len1, len2);
+    out.sync();
     
     
     //c. Rename file to file.new.
     //c. Rename file to file.new.
     final Path pnew = new Path(p + ".new");
     final Path pnew = new Path(p + ".new");
@@ -250,7 +250,7 @@ public class TestFileAppend3 extends junit.framework.TestCase {
       }
       }
       for(DatanodeInfo datanodeinfo : lb.getLocations()) {
       for(DatanodeInfo datanodeinfo : lb.getLocations()) {
         final DataNode dn = cluster.getDataNode(datanodeinfo.getIpcPort());
         final DataNode dn = cluster.getDataNode(datanodeinfo.getIpcPort());
-        final BlockMetaDataInfo metainfo = dn.getBlockMetaDataInfo(blk);
+        final Block metainfo = dn.data.getStoredBlock(blk.getBlockId());
         assertEquals(size, metainfo.getNumBytes());
         assertEquals(size, metainfo.getNumBytes());
       }
       }
     }
     }

+ 2 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCorruption.java

@@ -110,11 +110,11 @@ public class TestFileCorruption extends TestCase {
       
       
       // get the block
       // get the block
       File dataDir = new File(cluster.getDataDirectory(),
       File dataDir = new File(cluster.getDataDirectory(),
-          "data1/current");
+          "data1" + MiniDFSCluster.FINALIZED_DIR_NAME);
       Block blk = getBlock(dataDir);
       Block blk = getBlock(dataDir);
       if (blk == null) {
       if (blk == null) {
         blk = getBlock(new File(cluster.getDataDirectory(),
         blk = getBlock(new File(cluster.getDataDirectory(),
-          "dfs/data/data2/current"));
+          "dfs/data/data2" + MiniDFSCluster.FINALIZED_DIR_NAME));
       }
       }
       assertFalse(blk==null);
       assertFalse(blk==null);
 
 

+ 4 - 5
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -48,7 +48,6 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 
 
 
 
@@ -77,7 +76,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   // the datanodes.
   // the datanodes.
 
 
   // creates a file but does not close it
   // creates a file but does not close it
-  static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
+  public static FSDataOutputStream createFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
     throws IOException {
     System.out.println("createFile: Created " + name + " with " + repl + " replica.");
     System.out.println("createFile: Created " + name + " with " + repl + " replica.");
     FSDataOutputStream stm = fileSys.create(name, true,
     FSDataOutputStream stm = fileSys.create(name, true,
@@ -96,7 +95,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   //
   //
   // writes specified bytes to file.
   // writes specified bytes to file.
   //
   //
-  static void writeFile(FSDataOutputStream stm, int size) throws IOException {
+  public static void writeFile(FSDataOutputStream stm, int size) throws IOException {
     byte[] buffer = AppendTestUtil.randomBytes(seed, size);
     byte[] buffer = AppendTestUtil.randomBytes(seed, size);
     stm.write(buffer, 0, size);
     stm.write(buffer, 0, size);
   }
   }
@@ -447,9 +446,9 @@ public class TestFileCreation extends junit.framework.TestCase {
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
           + "The file has " + locations.locatedBlockCount() + " blocks.");
           + "The file has " + locations.locatedBlockCount() + " blocks.");
 
 
-      // add another block to the file
+      // add one block to the file
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(), 
       LocatedBlock location = client.getNamenode().addBlock(file1.toString(), 
-          client.clientName);
+          client.clientName, null);
       System.out.println("testFileCreationError2: "
       System.out.println("testFileCreationError2: "
           + "Added block " + location.getBlock());
           + "Added block " + location.getBlock());
 
 

+ 146 - 0
src/test/hdfs/org/apache/hadoop/hdfs/TestHFlush.java

@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FSDataInputStream;
+import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/** Class contains a set of tests to verify the correctness of 
+ * newly introduced {@link DFSClient#hflush()} method */
+public class TestHFlush {
+  private final String fName = "hflushtest.dat";
+  
+  /** The test uses {@link #doTheJob(Configuration, String, long, short)
+   * to write a file with a standard block size
+   */
+  @Test
+  public void hFlush_01() throws IOException {
+    doTheJob(new Configuration(), fName, AppendTestUtil.BLOCK_SIZE, (short)2);
+  }
+
+  /** The test uses {@link #doTheJob(Configuration, String, long, short)
+   * to write a file with a custom block size so the writes will be
+   * happening across block' boundaries
+   */
+  @Test
+  public void hFlush_02() throws IOException {
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 512;
+    int customBlockSize = customPerChecksumSize * 3;
+    // Modify defaul filesystem settings
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+
+    doTheJob(conf, fName, customBlockSize, (short)2);
+  }
+
+  /** The test uses {@link #doTheJob(Configuration, String, long, short)
+   * to write a file with a custom block size so the writes will be
+   * happening across block's and checksum' boundaries
+   */
+ @Test
+  public void hFlush_03() throws IOException {
+    Configuration conf = new Configuration();
+    int customPerChecksumSize = 400;
+    int customBlockSize = customPerChecksumSize * 3;
+    // Modify defaul filesystem settings
+    conf.setInt("io.bytes.per.checksum", customPerChecksumSize);
+    conf.setLong("dfs.block.size", customBlockSize);
+
+    doTheJob(conf, fName, customBlockSize, (short)2);
+  }
+
+  /**
+    The method starts new cluster with defined Configuration;
+    creates a file with specified block_size and writes 10 equal sections in it;
+    it also calls hflush() after each write and throws an IOException in case of 
+    an error.
+    @param conf cluster configuration
+    @param fileName of the file to be created and processed as required
+    @param block_size value to be used for the file's creation
+    @param replicas is the number of replicas
+    @throws IOException in case of any errors 
+   */
+  public static void doTheJob(Configuration conf, final String fileName,
+                              long block_size, short replicas) throws IOException {
+    byte[] fileContent;
+    final int SECTIONS = 10;
+
+    fileContent = AppendTestUtil.initBuffer(AppendTestUtil.FILE_SIZE);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, replicas, true, null);
+    // Make sure we work with DFS in order to utilize all its functionality
+    DistributedFileSystem fileSystem =
+        (DistributedFileSystem)cluster.getFileSystem();
+
+    FSDataInputStream is;
+    try {
+      Path path = new Path(fileName);
+      FSDataOutputStream stm = fileSystem.create(path, false, 4096, replicas,
+          block_size);
+      System.out.println("Created file " + fileName);
+
+      int tenth = AppendTestUtil.FILE_SIZE/SECTIONS;
+      int rounding = AppendTestUtil.FILE_SIZE - tenth * SECTIONS;
+      for (int i=0; i<SECTIONS; i++) {
+        System.out.println("Writing " + (tenth * i) + " to " + (tenth * (i+1)) + " section to file " + fileName);
+        // write to the file
+        stm.write(fileContent, tenth * i, tenth);
+        // Wait while hflush() pushes all packets through built pipeline
+        ((DFSClient.DFSOutputStream)stm.getWrappedStream()).hflush();
+        byte [] toRead = new byte[tenth];
+        byte [] expected = new byte[tenth];
+        System.arraycopy(fileContent, tenth * i, expected, 0, tenth);
+        // Open the same file for read. Need to create new reader after every write operation(!)
+        is = fileSystem.open(path);
+        is.read(toRead, tenth * i, tenth);
+        is.close();
+        checkData(toRead, 0, expected, "Partial verification");
+      }
+      System.out.println("Writing " + (tenth * SECTIONS) + " to " + (tenth * SECTIONS + rounding) + " section to file " + fileName);
+      stm.write(fileContent, tenth * SECTIONS, rounding);
+      stm.close();
+
+      assertEquals("File size doesn't match ", AppendTestUtil.FILE_SIZE, fileSystem.getFileStatus(path).getLen());
+      AppendTestUtil.checkFullFile(fileSystem, path, fileContent.length, fileContent, "hflush()");
+
+    } catch (IOException ioe) {
+      ioe.printStackTrace();
+      throw ioe;
+    } catch (Exception e) {
+      e.printStackTrace();
+    } finally {
+      fileSystem.close();
+      cluster.shutdown();
+    }
+  }
+  static void checkData(final byte[] actual, int from,
+                                final byte[] expected, String message) {
+    for (int idx = 0; idx < actual.length; idx++) {
+      assertEquals(message+" byte "+(from+idx)+" differs. expected "+
+                   expected[from+idx]+" actual "+actual[idx],
+                   expected[from+idx], actual[idx]);
+      actual[idx] = 0;
+    }
+  }
+}

+ 4 - 5
src/test/hdfs/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java

@@ -151,7 +151,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
       waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
       waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
 
 
       
       
-      Block[][] blocksList = cluster.getAllBlockReports();
+      Iterable<Block>[] blocksList = cluster.getAllBlockReports();
                     
                     
       
       
       cluster.shutdown();
       cluster.shutdown();
@@ -174,15 +174,14 @@ public class TestInjectionForSimulatedStorage extends TestCase {
       cluster.waitActive();
       cluster.waitActive();
       Set<Block> uniqueBlocks = new HashSet<Block>();
       Set<Block> uniqueBlocks = new HashSet<Block>();
       for (int i=0; i<blocksList.length; ++i) {
       for (int i=0; i<blocksList.length; ++i) {
-        for (int j=0; j < blocksList[i].length; ++j) {
-          uniqueBlocks.add(blocksList[i][j]);
+        for (Block b : blocksList[i]) {
+          uniqueBlocks.add(new Block(b));
         }
         }
       }
       }
       // Insert all the blocks in the first data node
       // Insert all the blocks in the first data node
       
       
       LOG.info("Inserting " + uniqueBlocks.size() + " blocks");
       LOG.info("Inserting " + uniqueBlocks.size() + " blocks");
-      Block[] blocks = uniqueBlocks.toArray(new Block[uniqueBlocks.size()]);
-      cluster.injectBlocks(0, blocks);
+      cluster.injectBlocks(0, uniqueBlocks);
       
       
       dfsClient = new DFSClient(new InetSocketAddress("localhost",
       dfsClient = new DFSClient(new InetSocketAddress("localhost",
                                   cluster.getNameNodePort()),
                                   cluster.getNameNodePort()),

+ 23 - 36
src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.Arrays;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -27,16 +26,15 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 
 
 public class TestLeaseRecovery extends junit.framework.TestCase {
 public class TestLeaseRecovery extends junit.framework.TestCase {
   static final int BLOCK_SIZE = 1024;
   static final int BLOCK_SIZE = 1024;
   static final short REPLICATION_NUM = (short)3;
   static final short REPLICATION_NUM = (short)3;
+  private static final long LEASE_PERIOD = 300L;
 
 
-  static void checkMetaInfo(Block b, InterDatanodeProtocol idp
+  static void checkMetaInfo(Block b, DataNode dn
       ) throws IOException {
       ) throws IOException {
-    TestInterDatanodeProtocol.checkMetaInfo(b, idp, null);
+    TestInterDatanodeProtocol.checkMetaInfo(b, dn);
   }
   }
   
   
   static int min(Integer... x) {
   static int min(Integer... x) {
@@ -49,6 +47,15 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
     return m;
     return m;
   }
   }
 
 
+  void waitLeaseRecovery(MiniDFSCluster cluster) {
+    cluster.setLeasePeriod(LEASE_PERIOD, LEASE_PERIOD);
+    // wait for the lease to expire
+    try {
+      Thread.sleep(2 * 3000);  // 2 heartbeat intervals
+    } catch (InterruptedException e) {
+    }
+  }
+
   /**
   /**
    * The following test first creates a file with a few blocks.
    * The following test first creates a file with a few blocks.
    * It randomly truncates the replica of the last block stored in each datanode.
    * It randomly truncates the replica of the last block stored in each datanode.
@@ -80,10 +87,8 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
       assertEquals(REPLICATION_NUM, datanodeinfos.length);
       assertEquals(REPLICATION_NUM, datanodeinfos.length);
 
 
       //connect to data nodes
       //connect to data nodes
-      InterDatanodeProtocol[] idps = new InterDatanodeProtocol[REPLICATION_NUM];
       DataNode[] datanodes = new DataNode[REPLICATION_NUM];
       DataNode[] datanodes = new DataNode[REPLICATION_NUM];
       for(int i = 0; i < REPLICATION_NUM; i++) {
       for(int i = 0; i < REPLICATION_NUM; i++) {
-        idps[i] = DataNode.createInterDataNodeProtocolProxy(datanodeinfos[i], conf);
         datanodes[i] = cluster.getDataNode(datanodeinfos[i].getIpcPort());
         datanodes[i] = cluster.getDataNode(datanodeinfos[i].getIpcPort());
         assertTrue(datanodes[i] != null);
         assertTrue(datanodes[i] != null);
       }
       }
@@ -92,44 +97,26 @@ public class TestLeaseRecovery extends junit.framework.TestCase {
       Block lastblock = locatedblock.getBlock();
       Block lastblock = locatedblock.getBlock();
       DataNode.LOG.info("newblocks=" + lastblock);
       DataNode.LOG.info("newblocks=" + lastblock);
       for(int i = 0; i < REPLICATION_NUM; i++) {
       for(int i = 0; i < REPLICATION_NUM; i++) {
-        checkMetaInfo(lastblock, idps[i]);
+        checkMetaInfo(lastblock, datanodes[i]);
       }
       }
 
 
-      //setup random block sizes 
-      int lastblocksize = ORG_FILE_SIZE % BLOCK_SIZE;
-      Integer[] newblocksizes = new Integer[REPLICATION_NUM];
-      for(int i = 0; i < REPLICATION_NUM; i++) {
-        newblocksizes[i] = AppendTestUtil.nextInt(lastblocksize);
-      }
-      DataNode.LOG.info("newblocksizes = " + Arrays.asList(newblocksizes)); 
-
-      //update blocks with random block sizes
-      Block[] newblocks = new Block[REPLICATION_NUM];
-      for(int i = 0; i < REPLICATION_NUM; i++) {
-        newblocks[i] = new Block(lastblock.getBlockId(), newblocksizes[i],
-            lastblock.getGenerationStamp());
-        idps[i].updateBlock(lastblock, newblocks[i], false);
-        checkMetaInfo(newblocks[i], idps[i]);
-      }
 
 
       DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName);
       DataNode.LOG.info("dfs.dfs.clientName=" + dfs.dfs.clientName);
       cluster.getNameNode().append(filestr, dfs.dfs.clientName);
       cluster.getNameNode().append(filestr, dfs.dfs.clientName);
 
 
-      //block synchronization
-      final int primarydatanodeindex = AppendTestUtil.nextInt(datanodes.length);
-      DataNode.LOG.info("primarydatanodeindex  =" + primarydatanodeindex);
-      DataNode primary = datanodes[primarydatanodeindex];
-      DataNode.LOG.info("primary.dnRegistration=" + primary.dnRegistration);
-      primary.recoverBlocks(new Block[]{lastblock}, new DatanodeInfo[][]{datanodeinfos}).join();
+      // expire lease to trigger block recovery.
+      waitLeaseRecovery(cluster);
 
 
-      BlockMetaDataInfo[] updatedmetainfo = new BlockMetaDataInfo[REPLICATION_NUM];
-      int minsize = min(newblocksizes);
-      long currentGS = cluster.getNamesystem().getGenerationStamp();
-      lastblock.setGenerationStamp(currentGS);
+      Block[] updatedmetainfo = new Block[REPLICATION_NUM];
+      long oldSize = lastblock.getNumBytes();
+      lastblock = TestInterDatanodeProtocol.getLastLocatedBlock(
+          dfs.dfs.getNamenode(), filestr).getBlock();
+      long currentGS = lastblock.getGenerationStamp();
       for(int i = 0; i < REPLICATION_NUM; i++) {
       for(int i = 0; i < REPLICATION_NUM; i++) {
-        updatedmetainfo[i] = idps[i].getBlockMetaDataInfo(lastblock);
+        updatedmetainfo[i] =
+          datanodes[i].data.getStoredBlock(lastblock.getBlockId());
         assertEquals(lastblock.getBlockId(), updatedmetainfo[i].getBlockId());
         assertEquals(lastblock.getBlockId(), updatedmetainfo[i].getBlockId());
-        assertEquals(minsize, updatedmetainfo[i].getNumBytes());
+        assertEquals(oldSize, updatedmetainfo[i].getNumBytes());
         assertEquals(currentGS, updatedmetainfo[i].getGenerationStamp());
         assertEquals(currentGS, updatedmetainfo[i].getGenerationStamp());
       }
       }
     }
     }

+ 5 - 4
src/test/hdfs/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

@@ -56,6 +56,7 @@ public class TestLeaseRecovery2 extends junit.framework.TestCase {
   //  conf.setInt("io.bytes.per.checksum", 16);
   //  conf.setInt("io.bytes.per.checksum", 16);
 
 
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
+    DistributedFileSystem dfs = null;
     byte[] actual = new byte[FILE_SIZE];
     byte[] actual = new byte[FILE_SIZE];
 
 
     try {
     try {
@@ -63,7 +64,7 @@ public class TestLeaseRecovery2 extends junit.framework.TestCase {
       cluster.waitActive();
       cluster.waitActive();
 
 
       //create a file
       //create a file
-      DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
+      dfs = (DistributedFileSystem)cluster.getFileSystem();
       // create a random file name
       // create a random file name
       String filestr = "/foo" + AppendTestUtil.nextInt();
       String filestr = "/foo" + AppendTestUtil.nextInt();
       System.out.println("filestr=" + filestr);
       System.out.println("filestr=" + filestr);
@@ -129,10 +130,9 @@ public class TestLeaseRecovery2 extends junit.framework.TestCase {
           + "Validating its contents now...");
           + "Validating its contents now...");
 
 
       // verify that file-size matches
       // verify that file-size matches
+      long fileSize = dfs.getFileStatus(filepath).getLen();
       assertTrue("File should be " + size + " bytes, but is actually " +
       assertTrue("File should be " + size + " bytes, but is actually " +
-                 " found to be " + dfs.getFileStatus(filepath).getLen() +
-                 " bytes",
-                 dfs.getFileStatus(filepath).getLen() == size);
+                 " found to be " + fileSize + " bytes", fileSize == size);
 
 
       // verify that there is enough data to read.
       // verify that there is enough data to read.
       System.out.println("File size is good. Now validating sizes from datanodes...");
       System.out.println("File size is good. Now validating sizes from datanodes...");
@@ -142,6 +142,7 @@ public class TestLeaseRecovery2 extends junit.framework.TestCase {
     }
     }
     finally {
     finally {
       try {
       try {
+        if(dfs != null) dfs.close();
         if (cluster != null) {cluster.shutdown();}
         if (cluster != null) {cluster.shutdown();}
       } catch (Exception e) {
       } catch (Exception e) {
         // ignore
         // ignore

+ 149 - 0
src/test/hdfs/org/apache/hadoop/hdfs/TestReadWhileWriting.java

@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.security.UnixUserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Test reading from hdfs while a file is being written. */
+public class TestReadWhileWriting {
+  {
+    ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  private static final String DIR = "/"
+      + TestReadWhileWriting.class.getSimpleName() + "/";
+  private static final int BLOCK_SIZE = 8192;
+  
+  /** Test reading while writing. */
+  @Test
+  public void testReadWhileWriting() throws Exception {
+    final Configuration conf = new Configuration();
+    //enable append
+    conf.setBoolean("dfs.support.append", true);
+
+    // create cluster
+    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    try {
+      //change the lease soft limit to 1 second.
+      final long leaseSoftLimit = 1000;
+      cluster.setLeasePeriod(leaseSoftLimit, FSConstants.LEASE_HARDLIMIT_PERIOD);
+
+      //wait for the cluster
+      cluster.waitActive();
+      final FileSystem fs = cluster.getFileSystem();
+      final Path p = new Path(DIR, "file1");
+      final int half = BLOCK_SIZE/2;
+
+      //a. On Machine M1, Create file. Write half block of data.
+      //   Invoke (DFSOutputStream).fsync() on the dfs file handle.
+      //   Do not close file yet.
+      {
+        final FSDataOutputStream out = fs.create(p, true,
+            fs.getConf().getInt("io.file.buffer.size", 4096),
+            (short)3, BLOCK_SIZE);
+        write(out, 0, half);
+
+        //hflush
+        ((DFSClient.DFSOutputStream)out.getWrappedStream()).hflush();
+      }
+
+      //b. On another machine M2, open file and verify that the half-block
+      //   of data can be read successfully.
+      checkFile(p, half, conf);
+
+      /* TODO: enable the following when append is done.
+      //c. On M1, append another half block of data.  Close file on M1.
+      {
+        //sleep to make sure the lease is expired the soft limit.
+        Thread.sleep(2*leaseSoftLimit);
+
+        FSDataOutputStream out = fs.append(p);
+        write(out, 0, half);
+        out.close();
+      }
+
+      //d. On M2, open file and read 1 block of data from it. Close file.
+      checkFile(p, 2*half, conf);
+      */
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  static private int userCount = 0;
+  //check the file
+  static void checkFile(Path p, int expectedsize, Configuration conf
+      ) throws IOException {
+    //open the file with another user account
+    final Configuration conf2 = new Configuration(conf);
+    final String username = UserGroupInformation.getCurrentUGI().getUserName()
+        + "_" + ++userCount;
+    UnixUserGroupInformation.saveToConf(conf2,
+        UnixUserGroupInformation.UGI_PROPERTY_NAME,
+        new UnixUserGroupInformation(username, new String[]{"supergroup"}));
+    final FileSystem fs = FileSystem.get(conf2);
+    final InputStream in = fs.open(p);
+
+    //Is the data available?
+    Assert.assertTrue(available(in, expectedsize));
+
+    //Able to read?
+    for(int i = 0; i < expectedsize; i++) {
+      Assert.assertEquals((byte)i, (byte)in.read());  
+    }
+
+    in.close();
+  }
+
+  /** Write something to a file */
+  private static void write(OutputStream out, int offset, int length
+      ) throws IOException {
+    final byte[] bytes = new byte[length];
+    for(int i = 0; i < length; i++) {
+      bytes[i] = (byte)(offset + i);
+    }
+    out.write(bytes);
+  }
+
+  /** Is the data available? */
+  private static boolean available(InputStream in, int expectedsize
+      ) throws IOException {
+    final int available = in.available();
+    System.out.println(" in.available()=" + available);
+    Assert.assertTrue(available >= 0);
+    Assert.assertTrue(available <= expectedsize);
+    return available == expectedsize;
+  }
+}
+

+ 9 - 4
src/test/hdfs/org/apache/hadoop/hdfs/TestRenameWhileOpen.java

@@ -36,6 +36,11 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
     ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
   }
   }
 
 
+  //TODO: un-comment checkFullFile once the lease recovery is done
+  private static void checkFullFile(FileSystem fs, Path p) throws IOException {
+    //TestFileCreation.checkFullFile(fs, p);
+  }
+
   /**
   /**
    * open /user/dir1/file1 /user/dir2/file2
    * open /user/dir1/file1 /user/dir2/file2
    * mkdir /user/dir3
    * mkdir /user/dir3
@@ -114,7 +119,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
       assertTrue(!fs.exists(file1));
       assertTrue(!fs.exists(file1));
       assertTrue(fs.exists(file2));
       assertTrue(fs.exists(file2));
       assertTrue(fs.exists(newfile));
       assertTrue(fs.exists(newfile));
-      TestFileCreation.checkFullFile(fs, newfile);
+      checkFullFile(fs, newfile);
     } finally {
     } finally {
       fs.close();
       fs.close();
       cluster.shutdown();
       cluster.shutdown();
@@ -186,7 +191,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
       assertTrue(!fs.exists(file1));
       assertTrue(!fs.exists(file1));
       assertTrue(fs.exists(file2));
       assertTrue(fs.exists(file2));
       assertTrue(fs.exists(newfile));
       assertTrue(fs.exists(newfile));
-      TestFileCreation.checkFullFile(fs, newfile);
+      checkFullFile(fs, newfile);
     } finally {
     } finally {
       fs.close();
       fs.close();
       cluster.shutdown();
       cluster.shutdown();
@@ -250,7 +255,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
       Path newfile = new Path("/user/dir2", "file1");
       Path newfile = new Path("/user/dir2", "file1");
       assertTrue(!fs.exists(file1));
       assertTrue(!fs.exists(file1));
       assertTrue(fs.exists(newfile));
       assertTrue(fs.exists(newfile));
-      TestFileCreation.checkFullFile(fs, newfile);
+      checkFullFile(fs, newfile);
     } finally {
     } finally {
       fs.close();
       fs.close();
       cluster.shutdown();
       cluster.shutdown();
@@ -312,7 +317,7 @@ public class TestRenameWhileOpen extends junit.framework.TestCase {
       Path newfile = new Path("/user", "dir2");
       Path newfile = new Path("/user", "dir2");
       assertTrue(!fs.exists(file1));
       assertTrue(!fs.exists(file1));
       assertTrue(fs.exists(newfile));
       assertTrue(fs.exists(newfile));
-      TestFileCreation.checkFullFile(fs, newfile);
+      checkFullFile(fs, newfile);
     } finally {
     } finally {
       fs.close();
       fs.close();
       cluster.shutdown();
       cluster.shutdown();

+ 3 - 2
src/test/hdfs/org/apache/hadoop/hdfs/TestReplication.java

@@ -291,7 +291,7 @@ public class TestReplication extends TestCase {
    * for under replicated blocks. 
    * for under replicated blocks. 
    * 
    * 
    * It creates a file with one block and replication of 4. It corrupts 
    * It creates a file with one block and replication of 4. It corrupts 
-   * two of the blocks and removes one of the replicas. Expected behaviour is
+   * two of the blocks and removes one of the replicas. Expected behavior is
    * that missing replica will be copied from one valid source.
    * that missing replica will be copied from one valid source.
    */
    */
   public void testPendingReplicationRetry() throws IOException {
   public void testPendingReplicationRetry() throws IOException {
@@ -341,7 +341,8 @@ public class TestReplication extends TestCase {
       
       
       int fileCount = 0;
       int fileCount = 0;
       for (int i=0; i<6; i++) {
       for (int i=0; i<6; i++) {
-        File blockFile = new File(baseDir, "data" + (i+1) + "/current/" + block);
+        File blockFile = new File(baseDir, "data" + (i+1) + 
+            MiniDFSCluster.FINALIZED_DIR_NAME + block);
         LOG.info("Checking for file " + blockFile);
         LOG.info("Checking for file " + blockFile);
         
         
         if (blockFile.exists()) {
         if (blockFile.exists()) {

+ 3 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.balancer;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 import java.util.Random;
 
 
@@ -169,7 +170,8 @@ public class TestBalancer extends TestCase {
     cluster.waitActive();
     cluster.waitActive();
     client = DFSClient.createNamenode(conf);
     client = DFSClient.createNamenode(conf);
 
 
-    cluster.injectBlocks(blocksDN);
+    for(int i = 0; i < blocksDN.length; i++)
+      cluster.injectBlocks(i, Arrays.asList(blocksDN[i]));
 
 
     long totalCapacity = 0L;
     long totalCapacity = 0L;
     for(long capacity:capacities) {
     for(long capacity:capacities) {

+ 204 - 29
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Random;
 import java.util.Random;
@@ -31,8 +32,12 @@ import javax.management.StandardMBean;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -77,11 +82,14 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
       nullCrcFileData[i+2] = nullCrcHeader[i];
       nullCrcFileData[i+2] = nullCrcHeader[i];
     }
     }
   }
   }
-  
-  private class BInfo { // information about a single block
+
+  // information about a single block
+  private class BInfo implements ReplicaInPipelineInterface {
     Block theBlock;
     Block theBlock;
     private boolean finalized = false; // if not finalized => ongoing creation
     private boolean finalized = false; // if not finalized => ongoing creation
     SimulatedOutputStream oStream = null;
     SimulatedOutputStream oStream = null;
+    private long bytesAcked;
+    private long bytesRcvd;
     BInfo(Block b, boolean forWriting) throws IOException {
     BInfo(Block b, boolean forWriting) throws IOException {
       theBlock = new Block(b);
       theBlock = new Block(b);
       if (theBlock.getNumBytes() < 0) {
       if (theBlock.getNumBytes() < 0) {
@@ -102,26 +110,27 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
       }
       }
     }
     }
 
 
-    synchronized long getGenerationStamp() {
+    synchronized public long getGenerationStamp() {
       return theBlock.getGenerationStamp();
       return theBlock.getGenerationStamp();
     }
     }
 
 
     synchronized void updateBlock(Block b) {
     synchronized void updateBlock(Block b) {
       theBlock.setGenerationStamp(b.getGenerationStamp());
       theBlock.setGenerationStamp(b.getGenerationStamp());
-      setlength(b.getNumBytes());
+      setNumBytes(b.getNumBytes());
+      setBytesOnDisk(b.getNumBytes());
     }
     }
     
     
-    synchronized long getlength() {
+    synchronized public long getNumBytes() {
       if (!finalized) {
       if (!finalized) {
-         return oStream.getLength();
+         return bytesRcvd;
       } else {
       } else {
         return theBlock.getNumBytes();
         return theBlock.getNumBytes();
       }
       }
     }
     }
 
 
-    synchronized void setlength(long length) {
+    synchronized public void setNumBytes(long length) {
       if (!finalized) {
       if (!finalized) {
-         oStream.setLength(length);
+         bytesRcvd = length;
       } else {
       } else {
         theBlock.setNumBytes(length);
         theBlock.setNumBytes(length);
       }
       }
@@ -170,7 +179,20 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
       oStream = null;
       oStream = null;
       return;
       return;
     }
     }
-    
+
+    synchronized void unfinalizeBlock() throws IOException {
+      if (!finalized) {
+        throw new IOException("Unfinalized a block that's not finalized "
+            + theBlock);
+      }
+      finalized = false;
+      oStream = new SimulatedOutputStream();
+      long blockLen = theBlock.getNumBytes();
+      oStream.setLength(blockLen);
+      bytesRcvd = blockLen;
+      bytesAcked = blockLen;
+    }
+
     SimulatedInputStream getMetaIStream() {
     SimulatedInputStream getMetaIStream() {
       return new SimulatedInputStream(nullCrcFileData);  
       return new SimulatedInputStream(nullCrcFileData);  
     }
     }
@@ -178,6 +200,64 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     synchronized boolean isFinalized() {
     synchronized boolean isFinalized() {
       return finalized;
       return finalized;
     }
     }
+
+    @Override
+    synchronized public BlockWriteStreams createStreams() throws IOException {
+      if (finalized) {
+        throw new IOException("Trying to write to a finalized replica "
+            + theBlock);
+      } else {
+        SimulatedOutputStream crcStream = new SimulatedOutputStream();
+        return new BlockWriteStreams(oStream, crcStream);
+      }
+    }
+
+    @Override
+    synchronized public long getBlockId() {
+      return theBlock.getBlockId();
+    }
+
+    @Override
+    synchronized public long getVisibleLength() {
+      return getBytesAcked();
+    }
+
+    @Override
+    public ReplicaState getState() {
+      return null;
+    }
+
+    @Override
+    synchronized public long getBytesAcked() {
+      if (finalized) {
+        return theBlock.getNumBytes();
+      } else {
+        return bytesAcked;
+      }
+    }
+
+    @Override
+    synchronized public void setBytesAcked(long bytesAcked) {
+      if (!finalized) {
+        this.bytesAcked = bytesAcked;
+      }
+    }
+
+    @Override
+    synchronized public long getBytesOnDisk() {
+      if (finalized) {
+        return theBlock.getNumBytes();
+      } else {
+        return oStream.getLength();
+      }
+    }
+
+    @Override
+    synchronized public void setBytesOnDisk(long bytesOnDisk) {
+      if (!finalized) {
+        oStream.setLength(bytesOnDisk);
+      }
+    }
   }
   }
   
   
   static private class SimulatedStorage {
   static private class SimulatedStorage {
@@ -243,10 +323,12 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     blockMap = new HashMap<Block,BInfo>(); 
     blockMap = new HashMap<Block,BInfo>(); 
   }
   }
 
 
-  public synchronized void injectBlocks(Block[] injectBlocks)
+  public synchronized void injectBlocks(Iterable<Block> injectBlocks)
                                             throws IOException {
                                             throws IOException {
     if (injectBlocks != null) {
     if (injectBlocks != null) {
+      int numInjectedBlocks = 0;
       for (Block b: injectBlocks) { // if any blocks in list is bad, reject list
       for (Block b: injectBlocks) { // if any blocks in list is bad, reject list
+        numInjectedBlocks++;
         if (b == null) {
         if (b == null) {
           throw new NullPointerException("Null blocks in block list");
           throw new NullPointerException("Null blocks in block list");
         }
         }
@@ -255,12 +337,12 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
         }
         }
       }
       }
       HashMap<Block, BInfo> oldBlockMap = blockMap;
       HashMap<Block, BInfo> oldBlockMap = blockMap;
-      blockMap = 
-          new HashMap<Block,BInfo>(injectBlocks.length + oldBlockMap.size());
+      blockMap = new HashMap<Block,BInfo>(
+          numInjectedBlocks + oldBlockMap.size());
       blockMap.putAll(oldBlockMap);
       blockMap.putAll(oldBlockMap);
       for (Block b: injectBlocks) {
       for (Block b: injectBlocks) {
           BInfo binfo = new BInfo(b, false);
           BInfo binfo = new BInfo(b, false);
-          blockMap.put(b, binfo);
+          blockMap.put(binfo.theBlock, binfo);
       }
       }
     }
     }
   }
   }
@@ -280,7 +362,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     }
     }
   }
   }
 
 
-  public synchronized Block[] getBlockReport() {
+  public synchronized BlockListAsLongs getBlockReport() {
     Block[] blockTable = new Block[blockMap.size()];
     Block[] blockTable = new Block[blockMap.size()];
     int count = 0;
     int count = 0;
     for (BInfo b : blockMap.values()) {
     for (BInfo b : blockMap.values()) {
@@ -291,7 +373,8 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     if (count != blockTable.length) {
     if (count != blockTable.length) {
       blockTable = Arrays.copyOf(blockTable, count);
       blockTable = Arrays.copyOf(blockTable, count);
     }
     }
-    return blockTable;
+    return new BlockListAsLongs(
+        new ArrayList<Block>(Arrays.asList(blockTable)), null);
   }
   }
 
 
   public long getCapacity() throws IOException {
   public long getCapacity() throws IOException {
@@ -311,7 +394,12 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     if (binfo == null) {
     if (binfo == null) {
       throw new IOException("Finalizing a non existing block " + b);
       throw new IOException("Finalizing a non existing block " + b);
     }
     }
-    return binfo.getlength();
+    return binfo.getNumBytes();
+  }
+
+  @Override
+  public Replica getReplica(long blockId) {
+    return blockMap.get(new Block(blockId));
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
@@ -322,7 +410,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
       return null;
       return null;
     }
     }
     b.setGenerationStamp(binfo.getGenerationStamp());
     b.setGenerationStamp(binfo.getGenerationStamp());
-    b.setNumBytes(binfo.getlength());
+    b.setNumBytes(binfo.getNumBytes());
     return b;
     return b;
   }
   }
 
 
@@ -350,7 +438,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
         DataNode.LOG.warn("Invalidate: Missing block");
         DataNode.LOG.warn("Invalidate: Missing block");
         continue;
         continue;
       }
       }
-      storage.free(binfo.getlength());
+      storage.free(binfo.getNumBytes());
       blockMap.remove(b);
       blockMap.remove(b);
     }
     }
       if (error) {
       if (error) {
@@ -380,21 +468,89 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     return getStorageInfo();
     return getStorageInfo();
   }
   }
 
 
-  public synchronized BlockWriteStreams writeToBlock(Block b, 
-                                            boolean isRecovery)
-                                            throws IOException {
+  @Override
+  public synchronized ReplicaInPipelineInterface append(Block b,
+      long newGS, long expectedBlockLen) throws IOException {
+    BInfo binfo = blockMap.get(b);
+    if (binfo == null || !binfo.isFinalized()) {
+      throw new ReplicaNotFoundException("Block " + b
+          + " is not valid, and cannot be appended to.");
+    }
+    binfo.unfinalizeBlock();
+    return binfo;
+  }
+
+  @Override
+  public synchronized ReplicaInPipelineInterface recoverAppend(Block b,
+      long newGS, long expectedBlockLen) throws IOException {
+    BInfo binfo = blockMap.get(b);
+    if (binfo == null) {
+      throw new ReplicaNotFoundException("Block " + b
+          + " is not valid, and cannot be appended to.");
+    }
+    if (binfo.isFinalized()) {
+      binfo.unfinalizeBlock();
+    }
+    blockMap.remove(b);
+    binfo.theBlock.setGenerationStamp(newGS);
+    blockMap.put(binfo.theBlock, binfo);
+    return binfo;
+  }
+
+  @Override
+  public void recoverClose(Block b, long newGS,
+      long expectedBlockLen) throws IOException {
+    BInfo binfo = blockMap.get(b);
+    if (binfo == null) {
+      throw new ReplicaNotFoundException("Block " + b
+          + " is not valid, and cannot be appended to.");
+    }
+    if (!binfo.isFinalized()) {
+      binfo.finalizeBlock(binfo.getNumBytes());
+    }
+    blockMap.remove(b);
+    binfo.theBlock.setGenerationStamp(newGS);
+    blockMap.put(binfo.theBlock, binfo);
+  }
+  
+  @Override
+  public synchronized ReplicaInPipelineInterface recoverRbw(Block b,
+      long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException {
+    BInfo binfo = blockMap.get(b);
+    if ( binfo == null) {
+      throw new ReplicaNotFoundException("Block " + b
+          + " does not exist, and cannot be appended to.");
+    }
+    if (binfo.isFinalized()) {
+      throw new ReplicaAlreadyExistsException("Block " + b
+          + " is valid, and cannot be written to.");
+    }
+    blockMap.remove(b);
+    binfo.theBlock.setGenerationStamp(newGS);
+    blockMap.put(binfo.theBlock, binfo);
+    return binfo;
+  }
+
+  @Override
+  public synchronized ReplicaInPipelineInterface createRbw(Block b) 
+  throws IOException {
+    return createTemporary(b);
+  }
+
+  @Override
+  public synchronized ReplicaInPipelineInterface createTemporary(Block b)
+      throws IOException {
     if (isValidBlock(b)) {
     if (isValidBlock(b)) {
-          throw new BlockAlreadyExistsException("Block " + b + 
+          throw new ReplicaAlreadyExistsException("Block " + b + 
               " is valid, and cannot be written to.");
               " is valid, and cannot be written to.");
       }
       }
     if (isBeingWritten(b)) {
     if (isBeingWritten(b)) {
-        throw new BlockAlreadyExistsException("Block " + b + 
+        throw new ReplicaAlreadyExistsException("Block " + b + 
             " is being written, and cannot be written to.");
             " is being written, and cannot be written to.");
     }
     }
-      BInfo binfo = new BInfo(b, true);
-      blockMap.put(b, binfo);
-      SimulatedOutputStream crcStream = new SimulatedOutputStream();
-      return new BlockWriteStreams(binfo.oStream, crcStream);
+    BInfo binfo = new BInfo(b, true);
+    blockMap.put(binfo.theBlock, binfo);
+    return binfo;
   }
   }
 
 
   public synchronized InputStream getBlockInputStream(Block b)
   public synchronized InputStream getBlockInputStream(Block b)
@@ -483,7 +639,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     if (binfo == null) {
     if (binfo == null) {
       throw new IOException("No such Block " + b );
       throw new IOException("No such Block " + b );
     }
     }
-    return binfo.getlength();
+    return binfo.getNumBytes();
   }
   }
 
 
   public synchronized void setChannelPosition(Block b, BlockWriteStreams stream, 
   public synchronized void setChannelPosition(Block b, BlockWriteStreams stream, 
@@ -493,7 +649,7 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
     if (binfo == null) {
     if (binfo == null) {
       throw new IOException("No such Block " + b );
       throw new IOException("No such Block " + b );
     }
     }
-    binfo.setlength(dataOffset);
+    binfo.setBytesOnDisk(dataOffset);
   }
   }
 
 
   /** 
   /** 
@@ -659,4 +815,23 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
   public boolean hasEnoughResource() {
   public boolean hasEnoughResource() {
     return true;
     return true;
   }
   }
+
+  @Override
+  public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
+  throws IOException {
+    return new ReplicaRecoveryInfo(rBlock.getBlock(), ReplicaState.FINALIZED);
+  }
+
+  @Override
+  public FinalizedReplica updateReplicaUnderRecovery(Block oldBlock,
+                                          long recoveryId,
+                                          long newlength) throws IOException {
+    return new FinalizedReplica(
+        oldBlock.getBlockId(), newlength, recoveryId, null, null);
+  }
+
+  @Override
+  public long getReplicaVisibleLength(Block block) throws IOException {
+    return block.getNumBytes();
+  }
 }
 }

+ 4 - 6
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -31,13 +31,11 @@ import junit.framework.TestCase;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
@@ -98,7 +96,7 @@ public class TestDataNodeVolumeFailure extends TestCase{
     // fail the volume
     // fail the volume
     // delete/make non-writable one of the directories (failed volume)
     // delete/make non-writable one of the directories (failed volume)
     data_fail = new File(dataDir, "data3");
     data_fail = new File(dataDir, "data3");
-    failedDir = new File(data_fail, "current");
+    failedDir = new File(data_fail, MiniDFSCluster.FINALIZED_DIR_NAME);
     if (failedDir.exists() &&
     if (failedDir.exists() &&
         //!FileUtil.fullyDelete(failedDir)
         //!FileUtil.fullyDelete(failedDir)
         !deteteBlocks(failedDir)
         !deteteBlocks(failedDir)
@@ -116,8 +114,8 @@ public class TestDataNodeVolumeFailure extends TestCase{
     
     
     // make sure a block report is sent 
     // make sure a block report is sent 
     DataNode dn = cluster.getDataNodes().get(1); //corresponds to dir data3
     DataNode dn = cluster.getDataNodes().get(1); //corresponds to dir data3
-    cluster.getNameNode().blockReport(dn.dnRegistration,
-        BlockListAsLongs.convertToArrayLongs(cluster.getBlockReport(1)));
+    long[] bReport = dn.getFSDataset().getBlockReport().getBlockListAsLongs();
+    cluster.getNameNode().blockReport(dn.dnRegistration, bReport);
 
 
     // verify number of blocks and files...
     // verify number of blocks and files...
     verify(filename, filesize);
     verify(filename, filesize);
@@ -302,7 +300,7 @@ public class TestDataNodeVolumeFailure extends TestCase{
     int total = 0;
     int total = 0;
     for(int i=0; i<dn_num; i++) {
     for(int i=0; i<dn_num; i++) {
       for(int j=1; j<=2; j++) {
       for(int j=1; j<=2; j++) {
-        File dir = new File(new File(dataDir, "data"+(2*i+j)), "current");
+        File dir = new File(dataDir, "data"+(2*i+j)+MiniDFSCluster.FINALIZED_DIR_NAME);
         if(dir == null) {
         if(dir == null) {
           System.out.println("dir is null for dn=" + i + " and data_dir=" + j);
           System.out.println("dir is null for dn=" + i + " and data_dir=" + j);
           continue;
           continue;

+ 204 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRestart.java

@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+import org.apache.hadoop.io.IOUtils;
+
+import org.junit.Test;
+import org.junit.Assert;
+
+/** Test if a datanode can correctly upgrade itself */
+public class TestDatanodeRestart {
+  // test finalized replicas persist across DataNode restarts
+  @Test public void testFinalizedReplicas() throws Exception {
+    // bring up a cluster of 3
+    Configuration conf = new Configuration();
+    conf.setLong("dfs.block.size", 1024L);
+    conf.setInt("dfs.write.packet.size", 512);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    cluster.waitActive();
+    FileSystem fs = cluster.getFileSystem();
+    try {
+      // test finalized replicas
+      final String TopDir = "/test";
+      DFSTestUtil util = new DFSTestUtil("TestCrcCorruption", 2, 3, 8*1024);
+      util.createFiles(fs, TopDir, (short)3);
+      util.waitReplication(fs, TopDir, (short)3);
+      util.checkFiles(fs, TopDir);
+      cluster.restartDataNodes();
+      cluster.waitActive();
+      util.checkFiles(fs, TopDir);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  // test rbw replicas persist across DataNode restarts
+  public void testRbwReplicas() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong("dfs.block.size", 1024L);
+    conf.setInt("dfs.write.packet.size", 512);
+    conf.setBoolean("dfs.support.append", true);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+    cluster.waitActive();
+    try {
+      testRbwReplicas(cluster, false);
+      testRbwReplicas(cluster, true);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+    
+  private void testRbwReplicas(MiniDFSCluster cluster, boolean isCorrupt) 
+  throws IOException {
+    FSDataOutputStream out = null;
+    FileSystem fs = cluster.getFileSystem();
+    final Path src = new Path("/test.txt");
+    try {
+      final int fileLen = 515;
+      // create some rbw replicas on disk
+      byte[] writeBuf = new byte[fileLen];
+      new Random().nextBytes(writeBuf);
+      out = fs.create(src);
+      out.write(writeBuf);
+      out.sync();
+      DataNode dn = cluster.getDataNodes().get(0);
+      for (FSVolume volume : ((FSDataset)dn.data).volumes.volumes) {
+        File currentDir = volume.getDir().getParentFile();
+        File rbwDir = new File(currentDir, "rbw");
+        for (File file : rbwDir.listFiles()) {
+          if (isCorrupt && Block.isBlockFilename(file)) {
+            new RandomAccessFile(file, "rw").setLength(fileLen-1); // corrupt
+          }
+        }
+      }
+      cluster.restartDataNodes();
+      cluster.waitActive();
+      dn = cluster.getDataNodes().get(0);
+
+      // check volumeMap: one rwr replica
+      ReplicasMap replicas = ((FSDataset)(dn.data)).volumeMap;
+      Assert.assertEquals(1, replicas.size());
+      ReplicaInfo replica = replicas.replicas().iterator().next();
+      Assert.assertEquals(ReplicaState.RWR, replica.getState());
+      if (isCorrupt) {
+        Assert.assertEquals((fileLen-1)/512*512, replica.getNumBytes());
+      } else {
+        Assert.assertEquals(fileLen, replica.getNumBytes());
+      }
+      dn.data.invalidate(new Block[]{replica});
+    } finally {
+      IOUtils.closeStream(out);
+      if (fs.exists(src)) {
+        fs.delete(src, false);
+      }
+      fs.close();
+    }      
+  }
+
+  // test recovering unlinked tmp replicas
+  @Test public void testRecoverReplicas() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong("dfs.block.size", 1024L);
+    conf.setInt("dfs.write.packet.size", 512);
+    conf.setBoolean("dfs.support.append", true);
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    cluster.waitActive();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      for (int i=0; i<4; i++) {
+        Path fileName = new Path("/test"+i);
+        DFSTestUtil.createFile(fs, fileName, 1, (short)1, 0L);
+        DFSTestUtil.waitReplication(fs, fileName, (short)1);
+      }
+      DataNode dn = cluster.getDataNodes().get(0);
+      Iterator<ReplicaInfo> replicasItor = 
+        ((FSDataset)dn.data).volumeMap.replicas().iterator();
+      ReplicaInfo replica = replicasItor.next();
+      createUnlinkTmpFile(replica, true, true); // rename block file
+      createUnlinkTmpFile(replica, false, true); // rename meta file
+      replica = replicasItor.next();
+      createUnlinkTmpFile(replica, true, false); // copy block file
+      createUnlinkTmpFile(replica, false, false); // copy meta file
+      replica = replicasItor.next();
+      createUnlinkTmpFile(replica, true, true); // rename block file
+      createUnlinkTmpFile(replica, false, false); // copy meta file
+
+      cluster.restartDataNodes();
+      cluster.waitActive();
+      dn = cluster.getDataNodes().get(0);
+
+      // check volumeMap: 4 finalized replica
+      Collection<ReplicaInfo> replicas = 
+        ((FSDataset)(dn.data)).volumeMap.replicas();
+      Assert.assertEquals(4, replicas.size());
+      replicasItor = replicas.iterator();
+      while (replicasItor.hasNext()) {
+        Assert.assertEquals(ReplicaState.FINALIZED, 
+            replicasItor.next().getState());
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  private static void createUnlinkTmpFile(ReplicaInfo replicaInfo, 
+      boolean changeBlockFile, 
+      boolean isRename) throws IOException {
+    File src;
+    if (changeBlockFile) {
+      src = replicaInfo.getBlockFile();
+    } else {
+      src = replicaInfo.getMetaFile();
+    }
+    File dst = FSDataset.getUnlinkTmpFile(src);
+    if (isRename) {
+      src.renameTo(dst);
+    } else {
+      FileInputStream in = new FileInputStream(src);
+      try {
+        FileOutputStream out = new FileOutputStream(dst);
+        try {
+          IOUtils.copyBytes(in, out, 1);
+        } finally {
+          out.close();
+        }
+      } finally {
+        in.close();
+      }
+    }
+  }
+}

+ 19 - 28
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -22,7 +22,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileChannel;
 import java.util.Random;
 import java.util.Random;
-import java.util.Map.Entry;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -67,17 +66,16 @@ public class TestDirectoryScanner extends TestCase {
   /** Truncate a block file */
   /** Truncate a block file */
   private long truncateBlockFile() throws IOException {
   private long truncateBlockFile() throws IOException {
     synchronized (fds) {
     synchronized (fds) {
-      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
-        Block b = entry.getKey();
-        File f = entry.getValue().getFile();
-        File mf = FSDataset.getMetaFile(f, b);
+      for (ReplicaInfo b : fds.volumeMap.replicas()) {
+        File f = b.getBlockFile();
+        File mf = b.getMetaFile();
         // Truncate a block file that has a corresponding metadata file
         // Truncate a block file that has a corresponding metadata file
         if (f.exists() && f.length() != 0 && mf.exists()) {
         if (f.exists() && f.length() != 0 && mf.exists()) {
           FileOutputStream s = new FileOutputStream(f);
           FileOutputStream s = new FileOutputStream(f);
           FileChannel channel = s.getChannel();
           FileChannel channel = s.getChannel();
           channel.truncate(0);
           channel.truncate(0);
           LOG.info("Truncated block file " + f.getAbsolutePath());
           LOG.info("Truncated block file " + f.getAbsolutePath());
-          return entry.getKey().getBlockId();
+          return b.getBlockId();
         }
         }
       }
       }
     }
     }
@@ -87,14 +85,13 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete a block file */
   /** Delete a block file */
   private long deleteBlockFile() {
   private long deleteBlockFile() {
     synchronized(fds) {
     synchronized(fds) {
-      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
-        Block b = entry.getKey();
-        File f = entry.getValue().getFile();
-        File mf = FSDataset.getMetaFile(f, b);
+      for (ReplicaInfo b : fds.volumeMap.replicas()) {
+        File f = b.getBlockFile();
+        File mf = b.getMetaFile();
         // Delete a block file that has corresponding metadata file
         // Delete a block file that has corresponding metadata file
         if (f.exists() && mf.exists() && f.delete()) {
         if (f.exists() && mf.exists() && f.delete()) {
           LOG.info("Deleting block file " + f.getAbsolutePath());
           LOG.info("Deleting block file " + f.getAbsolutePath());
-          return entry.getKey().getBlockId();
+          return b.getBlockId();
         }
         }
       }
       }
     }
     }
@@ -104,16 +101,12 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete block meta file */
   /** Delete block meta file */
   private long deleteMetaFile() {
   private long deleteMetaFile() {
     synchronized(fds) {
     synchronized(fds) {
-      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
-        Block b = entry.getKey();
-        String blkfile = entry.getValue().getFile().getAbsolutePath();
-        long genStamp = b.getGenerationStamp();
-        String metafile = FSDataset.getMetaFileName(blkfile, genStamp);
-        File file = new File(metafile);
+      for (ReplicaInfo b : fds.volumeMap.replicas()) {
+        File file = b.getMetaFile();
         // Delete a metadata file
         // Delete a metadata file
         if (file.exists() && file.delete()) {
         if (file.exists() && file.delete()) {
           LOG.info("Deleting metadata file " + file.getAbsolutePath());
           LOG.info("Deleting metadata file " + file.getAbsolutePath());
-          return entry.getKey().getBlockId();
+          return b.getBlockId();
         }
         }
       }
       }
     }
     }
@@ -324,23 +317,21 @@ public class TestDirectoryScanner extends TestCase {
   }
   }
 
 
   private void verifyAddition(long blockId, long genStamp, long size) {
   private void verifyAddition(long blockId, long genStamp, long size) {
-    Block memBlock = fds.getBlockKey(blockId);
-    assertNotNull(memBlock);
-    ReplicaInfo blockInfo;
+    final ReplicaInfo replicainfo;
     synchronized(fds) {
     synchronized(fds) {
-      blockInfo = fds.volumeMap.get(memBlock);
+      replicainfo = fds.getReplica(blockId);
     }
     }
-    assertNotNull(blockInfo);
+    assertNotNull(replicainfo);
 
 
     // Added block has the same file as the one created by the test
     // Added block has the same file as the one created by the test
     File file = new File(getBlockFile(blockId));
     File file = new File(getBlockFile(blockId));
-    assertEquals(file.getName(), blockInfo.getFile().getName());
+    assertEquals(file.getName(), replicainfo.getBlockFile().getName());
 
 
     // Generation stamp is same as that of created file
     // Generation stamp is same as that of created file
-    assertEquals(genStamp, memBlock.getGenerationStamp());
+    assertEquals(genStamp, replicainfo.getGenerationStamp());
 
 
     // File size matches
     // File size matches
-    assertEquals(size, memBlock.getNumBytes());
+    assertEquals(size, replicainfo.getNumBytes());
   }
   }
 
 
   private void verifyDeletion(long blockId) {
   private void verifyDeletion(long blockId) {
@@ -351,9 +342,9 @@ public class TestDirectoryScanner extends TestCase {
   }
   }
 
 
   private void verifyGenStamp(long blockId, long genStamp) {
   private void verifyGenStamp(long blockId, long genStamp) {
-    Block memBlock;
+    final Replica memBlock;
     synchronized(fds) {
     synchronized(fds) {
-      memBlock = fds.getBlockKey(blockId);
+      memBlock = fds.getReplica(blockId);
     }
     }
     assertNotNull(memBlock);
     assertNotNull(memBlock);
     assertEquals(genStamp, memBlock.getGenerationStamp());
     assertEquals(genStamp, memBlock.getGenerationStamp());

+ 13 - 15
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java

@@ -32,8 +32,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
 import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.Sender;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.AccessToken;
@@ -58,8 +61,8 @@ public class TestDiskError extends TestCase {
     FileSystem fs = cluster.getFileSystem();
     FileSystem fs = cluster.getFileSystem();
     final int dnIndex = 0;
     final int dnIndex = 0;
     String dataDir = cluster.getDataDirectory();
     String dataDir = cluster.getDataDirectory();
-    File dir1 = new File(new File(dataDir, "data"+(2*dnIndex+1)), "tmp");
-    File dir2 = new File(new File(dataDir, "data"+(2*dnIndex+2)), "tmp");
+    File dir1 = new File(new File(dataDir, "data"+(2*dnIndex+1)), "current/rbw");
+    File dir2 = new File(new File(dataDir, "data"+(2*dnIndex+2)), "current/rbw");
     try {
     try {
       // make the data directory of the first datanode to be readonly
       // make the data directory of the first datanode to be readonly
       assertTrue(dir1.setReadOnly());
       assertTrue(dir1.setReadOnly());
@@ -114,17 +117,12 @@ public class TestDiskError extends TestCase {
       DataOutputStream out = new DataOutputStream(
       DataOutputStream out = new DataOutputStream(
           s.getOutputStream());
           s.getOutputStream());
 
 
-      out.writeShort( DataTransferProtocol.DATA_TRANSFER_VERSION );
-      WRITE_BLOCK.write(out);
-      out.writeLong( block.getBlock().getBlockId());
-      out.writeLong( block.getBlock().getGenerationStamp() );
-      out.writeInt(1);
-      out.writeBoolean( false );       // recovery flag
-      Text.writeString( out, "" );
-      out.writeBoolean(false); // Not sending src node information
-      out.writeInt(0);
-      AccessToken.DUMMY_TOKEN.write(out);
-      
+      Sender.opWriteBlock(out, block.getBlock().getBlockId(), 
+          block.getBlock().getGenerationStamp(), 1, 
+          BlockConstructionStage.PIPELINE_SETUP_CREATE, 
+          0L, 0L, 0L, "", null, new DatanodeInfo[0], 
+          AccessToken.DUMMY_TOKEN);
+
       // write check header
       // write check header
       out.writeByte( 1 );
       out.writeByte( 1 );
       out.writeInt( 512 );
       out.writeInt( 512 );
@@ -136,8 +134,8 @@ public class TestDiskError extends TestCase {
       
       
       // the temporary block & meta files should be deleted
       // the temporary block & meta files should be deleted
       String dataDir = cluster.getDataDirectory();
       String dataDir = cluster.getDataDirectory();
-      File dir1 = new File(new File(dataDir, "data"+(2*sndNode+1)), "tmp");
-      File dir2 = new File(new File(dataDir, "data"+(2*sndNode+2)), "tmp");
+      File dir1 = new File(new File(dataDir, "data"+(2*sndNode+1)), "current/rbw");
+      File dir2 = new File(new File(dataDir, "data"+(2*sndNode+2)), "current/rbw");
       while (dir1.listFiles().length != 0 || dir2.listFiles().length != 0) {
       while (dir1.listFiles().length != 0 || dir2.listFiles().length != 0) {
         Thread.sleep(100);
         Thread.sleep(100);
       }
       }

+ 151 - 15
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java

@@ -21,33 +21,30 @@ import java.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
+import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.junit.Assert;
+import org.junit.Test;
 
 
 /**
 /**
  * This tests InterDataNodeProtocol for block handling. 
  * This tests InterDataNodeProtocol for block handling. 
  */
  */
 public class TestInterDatanodeProtocol extends junit.framework.TestCase {
 public class TestInterDatanodeProtocol extends junit.framework.TestCase {
-  public static void checkMetaInfo(Block b, InterDatanodeProtocol idp,
-      DataBlockScanner scanner) throws IOException {
-    BlockMetaDataInfo metainfo = idp.getBlockMetaDataInfo(b);
+  public static void checkMetaInfo(Block b, DataNode dn) throws IOException {
+    Block metainfo = dn.data.getStoredBlock(b.getBlockId());
     assertEquals(b.getBlockId(), metainfo.getBlockId());
     assertEquals(b.getBlockId(), metainfo.getBlockId());
     assertEquals(b.getNumBytes(), metainfo.getNumBytes());
     assertEquals(b.getNumBytes(), metainfo.getNumBytes());
-    if (scanner != null) {
-      assertEquals(scanner.getLastScanTime(b),
-          metainfo.getLastScanTime());
-    }
   }
   }
 
 
   public static LocatedBlock getLastLocatedBlock(
   public static LocatedBlock getLastLocatedBlock(
@@ -99,16 +96,155 @@ public class TestInterDatanodeProtocol extends junit.framework.TestCase {
       //verify BlockMetaDataInfo
       //verify BlockMetaDataInfo
       Block b = locatedblock.getBlock();
       Block b = locatedblock.getBlock();
       InterDatanodeProtocol.LOG.info("b=" + b + ", " + b.getClass());
       InterDatanodeProtocol.LOG.info("b=" + b + ", " + b.getClass());
-      checkMetaInfo(b, idp, datanode.blockScanner);
+      checkMetaInfo(b, datanode);
+      long recoveryId = b.getGenerationStamp() + 1;
+      idp.initReplicaRecovery(
+          new RecoveringBlock(b, locatedblock.getLocations(), recoveryId));
 
 
       //verify updateBlock
       //verify updateBlock
       Block newblock = new Block(
       Block newblock = new Block(
           b.getBlockId(), b.getNumBytes()/2, b.getGenerationStamp()+1);
           b.getBlockId(), b.getNumBytes()/2, b.getGenerationStamp()+1);
-      idp.updateBlock(b, newblock, false);
-      checkMetaInfo(newblock, idp, datanode.blockScanner);
+      idp.updateReplicaUnderRecovery(b, recoveryId, newblock.getNumBytes());
+      checkMetaInfo(newblock, datanode);
     }
     }
     finally {
     finally {
       if (cluster != null) {cluster.shutdown();}
       if (cluster != null) {cluster.shutdown();}
     }
     }
   }
   }
+
+  private static ReplicaInfo createReplicaInfo(Block b) {
+    return new ReplicaBeingWritten(b.getBlockId(), b.getGenerationStamp(),
+        null, null);
+  }
+
+  private static void assertEquals(ReplicaInfo originalInfo, ReplicaRecoveryInfo recoveryInfo) {
+    Assert.assertEquals(originalInfo.getBlockId(), recoveryInfo.getBlockId());
+    Assert.assertEquals(originalInfo.getGenerationStamp(), recoveryInfo.getGenerationStamp());
+    Assert.assertEquals(originalInfo.getBytesOnDisk(), recoveryInfo.getNumBytes());
+    Assert.assertEquals(originalInfo.getState(), recoveryInfo.getOriginalReplicaState());
+  }
+
+  /** Test {@link FSDataset#initReplicaRecovery(ReplicasMap, Block, long)} */
+  @Test
+  public void testInitReplicaRecovery() throws IOException {
+    final long firstblockid = 10000L;
+    final long gs = 7777L;
+    final long length = 22L;
+    final ReplicasMap map = new ReplicasMap();
+    final Block[] blocks = new Block[5];
+    for(int i = 0; i < blocks.length; i++) {
+      blocks[i] = new Block(firstblockid + i, length, gs);
+      map.add(createReplicaInfo(blocks[i]));
+    }
+    
+    { 
+      //normal case
+      final Block b = blocks[0];
+      final ReplicaInfo originalInfo = map.get(b);
+
+      final long recoveryid = gs + 1;
+      final ReplicaRecoveryInfo recoveryInfo = FSDataset.initReplicaRecovery(map, blocks[0], recoveryid);
+      assertEquals(originalInfo, recoveryInfo);
+
+      final ReplicaUnderRecovery updatedInfo = (ReplicaUnderRecovery)map.get(b);
+      Assert.assertEquals(originalInfo.getBlockId(), updatedInfo.getBlockId());
+      Assert.assertEquals(recoveryid, updatedInfo.getRecoveryID());
+
+      //recover one more time 
+      final long recoveryid2 = gs + 2;
+      final ReplicaRecoveryInfo recoveryInfo2 = FSDataset.initReplicaRecovery(map, blocks[0], recoveryid2);
+      assertEquals(originalInfo, recoveryInfo2);
+
+      final ReplicaUnderRecovery updatedInfo2 = (ReplicaUnderRecovery)map.get(b);
+      Assert.assertEquals(originalInfo.getBlockId(), updatedInfo2.getBlockId());
+      Assert.assertEquals(recoveryid2, updatedInfo2.getRecoveryID());
+      
+      //case RecoveryInProgressException
+      try {
+        FSDataset.initReplicaRecovery(map, b, recoveryid);
+        Assert.fail();
+      }
+      catch(RecoveryInProgressException ripe) {
+        System.out.println("GOOD: getting " + ripe);
+      }
+    }
+
+    { //replica not found
+      final long recoveryid = gs + 1;
+      final Block b = new Block(firstblockid - 1, length, gs);
+      ReplicaRecoveryInfo r = FSDataset.initReplicaRecovery(map, b, recoveryid);
+      Assert.assertNull("Data-node should not have this replica.", r);
+    }
+    
+    { //case "THIS IS NOT SUPPOSED TO HAPPEN"
+      final long recoveryid = gs - 1;
+      final Block b = new Block(firstblockid + 1, length, gs);
+      try {
+        FSDataset.initReplicaRecovery(map, b, recoveryid);
+        Assert.fail();
+      }
+      catch(IOException ioe) {
+        System.out.println("GOOD: getting " + ioe);
+      }
+    }
+
+  }
+
+  /** Test {@link FSDataset#updateReplicaUnderRecovery(ReplicaUnderRecovery, long, long)} */
+  @Test
+  public void testUpdateReplicaUnderRecovery() throws IOException {
+    final Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+
+    try {
+      cluster = new MiniDFSCluster(conf, 3, true, null);
+      cluster.waitActive();
+
+      //create a file
+      DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
+      String filestr = "/foo";
+      Path filepath = new Path(filestr);
+      DFSTestUtil.createFile(dfs, filepath, 1024L, (short)3, 0L);
+
+      //get block info
+      final LocatedBlock locatedblock = getLastLocatedBlock(
+          dfs.getClient().getNamenode(), filestr);
+      final DatanodeInfo[] datanodeinfo = locatedblock.getLocations();
+      Assert.assertTrue(datanodeinfo.length > 0);
+
+      //get DataNode and FSDataset objects
+      final DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
+      Assert.assertTrue(datanode != null);
+      Assert.assertTrue(datanode.data instanceof FSDataset);
+      final FSDataset fsdataset = (FSDataset)datanode.data;
+
+      //initReplicaRecovery
+      final Block b = locatedblock.getBlock();
+      final long recoveryid = b.getGenerationStamp() + 1;
+      final long newlength = b.getNumBytes() - 1;
+      FSDataset.initReplicaRecovery(fsdataset.volumeMap, b, recoveryid);
+
+      //check replica
+      final ReplicaInfo replica = fsdataset.getReplica(b.getBlockId());
+      Assert.assertTrue(replica instanceof ReplicaUnderRecovery);
+      final ReplicaUnderRecovery rur = (ReplicaUnderRecovery)replica;
+
+      //check meta data before update
+      FSDataset.checkReplicaFiles(rur);
+
+      //update
+      final ReplicaInfo finalized = 
+        (ReplicaInfo)fsdataset.updateReplicaUnderRecovery(
+            rur, recoveryid, newlength);
+
+      //check meta data after update
+      FSDataset.checkReplicaFiles(finalized);
+      Assert.assertEquals(b.getBlockId(), finalized.getBlockId());
+      Assert.assertEquals(recoveryid, finalized.getGenerationStamp());
+      Assert.assertEquals(newlength, finalized.getNumBytes());
+
+    } finally {
+      if (cluster != null) cluster.shutdown();
+    }
+  }
 }
 }

+ 31 - 24
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java

@@ -25,8 +25,10 @@ import junit.framework.TestCase;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface.BlockWriteStreams;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 
 
 /**
 /**
@@ -62,14 +64,19 @@ public class TestSimulatedFSDataset extends TestCase {
     int bytesAdded = 0;
     int bytesAdded = 0;
     for (int i = startingBlockId; i < startingBlockId+NUMBLOCKS; ++i) {
     for (int i = startingBlockId; i < startingBlockId+NUMBLOCKS; ++i) {
       Block b = new Block(i, 0, 0); // we pass expected len as zero, - fsdataset should use the sizeof actual data written
       Block b = new Block(i, 0, 0); // we pass expected len as zero, - fsdataset should use the sizeof actual data written
-      OutputStream dataOut  = fsdataset.writeToBlock(b, false).dataOut;
-      assertEquals(0, fsdataset.getLength(b));
-      for (int j=1; j <= blockIdToLen(i); ++j) {
-        dataOut.write(j);
-        assertEquals(j, fsdataset.getLength(b)); // correct length even as we write
-        bytesAdded++;
+      ReplicaInPipelineInterface bInfo = fsdataset.createRbw(b);
+      BlockWriteStreams out = bInfo.createStreams();
+      try {
+        OutputStream dataOut  = out.dataOut;
+        assertEquals(0, fsdataset.getLength(b));
+        for (int j=1; j <= blockIdToLen(i); ++j) {
+          dataOut.write(j);
+          assertEquals(j, bInfo.getBytesOnDisk()); // correct length even as we write
+          bytesAdded++;
+        }
+      } finally {
+        out.close();
       }
       }
-      dataOut.close();
       b.setNumBytes(blockIdToLen(i));
       b.setNumBytes(blockIdToLen(i));
       fsdataset.finalizeBlock(b);
       fsdataset.finalizeBlock(b);
       assertEquals(blockIdToLen(i), fsdataset.getLength(b));
       assertEquals(blockIdToLen(i), fsdataset.getLength(b));
@@ -139,24 +146,24 @@ public class TestSimulatedFSDataset extends TestCase {
 
 
 
 
   public void testGetBlockReport() throws IOException {
   public void testGetBlockReport() throws IOException {
-    FSDatasetInterface fsdataset = new SimulatedFSDataset(conf); 
-    Block[] blockReport = fsdataset.getBlockReport();
-    assertEquals(0, blockReport.length);
+    SimulatedFSDataset fsdataset = new SimulatedFSDataset(conf); 
+    BlockListAsLongs blockReport = fsdataset.getBlockReport();
+    assertEquals(0, blockReport.getNumberOfBlocks());
     int bytesAdded = addSomeBlocks(fsdataset);
     int bytesAdded = addSomeBlocks(fsdataset);
     blockReport = fsdataset.getBlockReport();
     blockReport = fsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS, blockReport.length);
+    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
     for (Block b: blockReport) {
     for (Block b: blockReport) {
       assertNotNull(b);
       assertNotNull(b);
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
     }
     }
   }
   }
   public void testInjectionEmpty() throws IOException {
   public void testInjectionEmpty() throws IOException {
-    FSDatasetInterface fsdataset = new SimulatedFSDataset(conf); 
-    Block[] blockReport = fsdataset.getBlockReport();
-    assertEquals(0, blockReport.length);
+    SimulatedFSDataset fsdataset = new SimulatedFSDataset(conf); 
+    BlockListAsLongs blockReport = fsdataset.getBlockReport();
+    assertEquals(0, blockReport.getNumberOfBlocks());
     int bytesAdded = addSomeBlocks(fsdataset);
     int bytesAdded = addSomeBlocks(fsdataset);
     blockReport = fsdataset.getBlockReport();
     blockReport = fsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS, blockReport.length);
+    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
     for (Block b: blockReport) {
     for (Block b: blockReport) {
       assertNotNull(b);
       assertNotNull(b);
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
@@ -169,7 +176,7 @@ public class TestSimulatedFSDataset extends TestCase {
     SimulatedFSDataset sfsdataset = new SimulatedFSDataset(conf);
     SimulatedFSDataset sfsdataset = new SimulatedFSDataset(conf);
     sfsdataset.injectBlocks(blockReport);
     sfsdataset.injectBlocks(blockReport);
     blockReport = sfsdataset.getBlockReport();
     blockReport = sfsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS, blockReport.length);
+    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
     for (Block b: blockReport) {
     for (Block b: blockReport) {
       assertNotNull(b);
       assertNotNull(b);
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
@@ -180,13 +187,13 @@ public class TestSimulatedFSDataset extends TestCase {
   }
   }
 
 
   public void testInjectionNonEmpty() throws IOException {
   public void testInjectionNonEmpty() throws IOException {
-    FSDatasetInterface fsdataset = new SimulatedFSDataset(conf); 
+    SimulatedFSDataset fsdataset = new SimulatedFSDataset(conf); 
     
     
-    Block[] blockReport = fsdataset.getBlockReport();
-    assertEquals(0, blockReport.length);
+    BlockListAsLongs blockReport = fsdataset.getBlockReport();
+    assertEquals(0, blockReport.getNumberOfBlocks());
     int bytesAdded = addSomeBlocks(fsdataset);
     int bytesAdded = addSomeBlocks(fsdataset);
     blockReport = fsdataset.getBlockReport();
     blockReport = fsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS, blockReport.length);
+    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
     for (Block b: blockReport) {
     for (Block b: blockReport) {
       assertNotNull(b);
       assertNotNull(b);
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
@@ -201,13 +208,13 @@ public class TestSimulatedFSDataset extends TestCase {
     // Add come blocks whose block ids do not conflict with
     // Add come blocks whose block ids do not conflict with
     // the ones we are going to inject.
     // the ones we are going to inject.
     bytesAdded += addSomeBlocks(sfsdataset, NUMBLOCKS+1);
     bytesAdded += addSomeBlocks(sfsdataset, NUMBLOCKS+1);
-    Block[] blockReport2 = sfsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS, blockReport.length);
+    BlockListAsLongs blockReport2 = sfsdataset.getBlockReport();
+    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
     blockReport2 = sfsdataset.getBlockReport();
     blockReport2 = sfsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS, blockReport.length);
+    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
     sfsdataset.injectBlocks(blockReport);
     sfsdataset.injectBlocks(blockReport);
     blockReport = sfsdataset.getBlockReport();
     blockReport = sfsdataset.getBlockReport();
-    assertEquals(NUMBLOCKS*2, blockReport.length);
+    assertEquals(NUMBLOCKS*2, blockReport.getNumberOfBlocks());
     for (Block b: blockReport) {
     for (Block b: blockReport) {
       assertNotNull(b);
       assertNotNull(b);
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
       assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());

+ 431 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestWriteToReplica.java

@@ -0,0 +1,431 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Test if FSDataset#append, writeToRbw, and writeToTmp */
+public class TestWriteToReplica {
+  final private static Block[] blocks = new Block[] {
+    new Block(1, 1, 2001), new Block(2, 1, 2002), 
+    new Block(3, 1, 2003), new Block(4, 1, 2004),
+    new Block(5, 1, 2005), new Block(6, 1, 2006)
+  };
+  final private static int FINALIZED = 0;
+  final private static int TEMPORARY = 1;
+  final private static int RBW = 2;
+  final private static int RWR = 3;
+  final private static int RUR = 4;
+  final private static int NON_EXISTENT = 5;
+  
+  // test close
+  @Test
+  public void testClose() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster(new Configuration(), 1, true, null);
+    try {
+      cluster.waitActive();
+      DataNode dn = cluster.getDataNodes().get(0);
+      FSDataset dataSet = (FSDataset)dn.data;
+
+      // set up replicasMap
+      setup(dataSet);
+
+      // test close
+      testClose(dataSet);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  // test append
+  @Test
+  public void testAppend() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster(new Configuration(), 1, true, null);
+    try {
+      cluster.waitActive();
+      DataNode dn = cluster.getDataNodes().get(0);
+      FSDataset dataSet = (FSDataset)dn.data;
+
+      // set up replicasMap
+      setup(dataSet);
+
+      // test append
+      testAppend(dataSet);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  // test writeToRbw
+  @Test
+  public void testWriteToRbw() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster(new Configuration(), 1, true, null);
+    try {
+      cluster.waitActive();
+      DataNode dn = cluster.getDataNodes().get(0);
+      FSDataset dataSet = (FSDataset)dn.data;
+
+      // set up replicasMap
+      setup(dataSet);
+
+      // test writeToRbw
+      testWriteToRbw(dataSet);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  // test writeToTemporary
+  @Test
+  public void testWriteToTempoary() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster(new Configuration(), 1, true, null);
+    try {
+      cluster.waitActive();
+      DataNode dn = cluster.getDataNodes().get(0);
+      FSDataset dataSet = (FSDataset)dn.data;
+
+      // set up replicasMap
+      setup(dataSet);
+
+      // test writeToTemporary
+      testWriteToTemporary(dataSet);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  private void setup(FSDataset dataSet) throws IOException {
+    // setup replicas map
+    ReplicasMap replicasMap = dataSet.volumeMap;
+    FSVolume vol = dataSet.volumes.getNextVolume(0);
+    ReplicaInfo replicaInfo = new FinalizedReplica(
+        blocks[FINALIZED], vol, vol.getDir());
+    replicasMap.add(replicaInfo);
+    replicaInfo.getBlockFile().createNewFile();
+    replicaInfo.getMetaFile().createNewFile();
+    
+    replicasMap.add(new ReplicaInPipeline(
+        blocks[TEMPORARY].getBlockId(),
+        blocks[TEMPORARY].getGenerationStamp(), vol, 
+        vol.createTmpFile(blocks[TEMPORARY]).getParentFile()));
+    
+    replicaInfo = new ReplicaBeingWritten(blocks[RBW], vol, 
+        vol.createRbwFile(blocks[RBW]).getParentFile(), null);
+    replicasMap.add(replicaInfo);
+    replicaInfo.getBlockFile().createNewFile();
+    replicaInfo.getMetaFile().createNewFile();
+    
+    replicasMap.add(new ReplicaWaitingToBeRecovered(blocks[RWR], vol, 
+        vol.createRbwFile(blocks[RWR]).getParentFile()));
+    replicasMap.add(new ReplicaUnderRecovery(
+        new FinalizedReplica(blocks[RUR], vol, vol.getDir()), 2007));    
+  }
+  
+  private void testAppend(FSDataset dataSet) throws IOException {
+    long newGS = blocks[FINALIZED].getGenerationStamp()+1;
+    dataSet.append(blocks[FINALIZED], newGS, 
+        blocks[FINALIZED].getNumBytes());  // successful
+    blocks[FINALIZED].setGenerationStamp(newGS);
+    
+    try {
+      dataSet.append(blocks[TEMPORARY], blocks[TEMPORARY].getGenerationStamp()+1, 
+          blocks[TEMPORARY].getNumBytes());
+      Assert.fail("Should not have appended to a temporary replica " 
+          + blocks[TEMPORARY]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertEquals(ReplicaNotFoundException.UNFINALIZED_REPLICA +
+          blocks[TEMPORARY], e.getMessage());
+    }
+
+    try {
+      dataSet.append(blocks[RBW], blocks[RBW].getGenerationStamp()+1,
+          blocks[RBW].getNumBytes());
+      Assert.fail("Should not have appended to an RBW replica" + blocks[RBW]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertEquals(ReplicaNotFoundException.UNFINALIZED_REPLICA +
+          blocks[RBW], e.getMessage());
+    }
+
+    try {
+      dataSet.append(blocks[RWR], blocks[RWR].getGenerationStamp()+1,
+          blocks[RBW].getNumBytes());
+      Assert.fail("Should not have appended to an RWR replica" + blocks[RWR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertEquals(ReplicaNotFoundException.UNFINALIZED_REPLICA +
+          blocks[RWR], e.getMessage());
+    }
+
+    try {
+      dataSet.append(blocks[RUR], blocks[RUR].getGenerationStamp()+1,
+          blocks[RUR].getNumBytes());
+      Assert.fail("Should not have appended to an RUR replica" + blocks[RUR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertEquals(ReplicaNotFoundException.UNFINALIZED_REPLICA +
+          blocks[RUR], e.getMessage());
+    }
+
+    try {
+      dataSet.append(blocks[NON_EXISTENT], 
+          blocks[NON_EXISTENT].getGenerationStamp(), 
+          blocks[NON_EXISTENT].getNumBytes());
+      Assert.fail("Should not have appended to a non-existent replica " + 
+          blocks[NON_EXISTENT]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertEquals(ReplicaNotFoundException.NON_EXISTENT_REPLICA + 
+          blocks[NON_EXISTENT], e.getMessage());
+    }
+    
+    newGS = blocks[FINALIZED].getGenerationStamp()+1;
+    dataSet.recoverAppend(blocks[FINALIZED], newGS, 
+        blocks[FINALIZED].getNumBytes());  // successful
+    blocks[FINALIZED].setGenerationStamp(newGS);
+    
+    try {
+      dataSet.recoverAppend(blocks[TEMPORARY], blocks[TEMPORARY].getGenerationStamp()+1, 
+          blocks[TEMPORARY].getNumBytes());
+      Assert.fail("Should not have appended to a temporary replica " 
+          + blocks[TEMPORARY]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.UNFINALIZED_AND_NONRBW_REPLICA));
+    }
+
+    newGS = blocks[RBW].getGenerationStamp()+1;
+    dataSet.recoverAppend(blocks[RBW], newGS, blocks[RBW].getNumBytes());
+    blocks[RBW].setGenerationStamp(newGS);
+
+    try {
+      dataSet.recoverAppend(blocks[RWR], blocks[RWR].getGenerationStamp()+1,
+          blocks[RBW].getNumBytes());
+      Assert.fail("Should not have appended to an RWR replica" + blocks[RWR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.UNFINALIZED_AND_NONRBW_REPLICA));
+    }
+
+    try {
+      dataSet.recoverAppend(blocks[RUR], blocks[RUR].getGenerationStamp()+1,
+          blocks[RUR].getNumBytes());
+      Assert.fail("Should not have appended to an RUR replica" + blocks[RUR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.UNFINALIZED_AND_NONRBW_REPLICA));
+    }
+
+    try {
+      dataSet.recoverAppend(blocks[NON_EXISTENT], 
+          blocks[NON_EXISTENT].getGenerationStamp(), 
+          blocks[NON_EXISTENT].getNumBytes());
+      Assert.fail("Should not have appended to a non-existent replica " + 
+          blocks[NON_EXISTENT]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.NON_EXISTENT_REPLICA));
+    }
+  }
+
+  private void testClose(FSDataset dataSet) throws IOException {
+    long newGS = blocks[FINALIZED].getGenerationStamp()+1;
+    dataSet.recoverClose(blocks[FINALIZED], newGS, 
+        blocks[FINALIZED].getNumBytes());  // successful
+    blocks[FINALIZED].setGenerationStamp(newGS);
+    
+    try {
+      dataSet.recoverClose(blocks[TEMPORARY], blocks[TEMPORARY].getGenerationStamp()+1, 
+          blocks[TEMPORARY].getNumBytes());
+      Assert.fail("Should not have recovered close a temporary replica " 
+          + blocks[TEMPORARY]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.UNFINALIZED_AND_NONRBW_REPLICA));
+    }
+
+    newGS = blocks[RBW].getGenerationStamp()+1;
+    dataSet.recoverClose(blocks[RBW], newGS, blocks[RBW].getNumBytes());
+    blocks[RBW].setGenerationStamp(newGS);
+
+    try {
+      dataSet.recoverClose(blocks[RWR], blocks[RWR].getGenerationStamp()+1,
+          blocks[RBW].getNumBytes());
+      Assert.fail("Should not have recovered close an RWR replica" + blocks[RWR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.UNFINALIZED_AND_NONRBW_REPLICA));
+    }
+
+    try {
+      dataSet.recoverClose(blocks[RUR], blocks[RUR].getGenerationStamp()+1,
+          blocks[RUR].getNumBytes());
+      Assert.fail("Should not have recovered close an RUR replica" + blocks[RUR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.UNFINALIZED_AND_NONRBW_REPLICA));
+    }
+
+    try {
+      dataSet.recoverClose(blocks[NON_EXISTENT], 
+          blocks[NON_EXISTENT].getGenerationStamp(), 
+          blocks[NON_EXISTENT].getNumBytes());
+      Assert.fail("Should not have recovered close a non-existent replica " + 
+          blocks[NON_EXISTENT]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.NON_EXISTENT_REPLICA));
+    }
+  }
+  
+  private void testWriteToRbw(FSDataset dataSet) throws IOException {
+    try {
+      dataSet.recoverRbw(blocks[FINALIZED],
+          blocks[FINALIZED].getGenerationStamp()+1,
+          0L, blocks[FINALIZED].getNumBytes());
+      Assert.fail("Should not have recovered a finalized replica " +
+          blocks[FINALIZED]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.NON_RBW_REPLICA));
+    }
+ 
+    try {
+      dataSet.createRbw(blocks[FINALIZED]);
+      Assert.fail("Should not have created a replica that's already " +
+      		"finalized " + blocks[FINALIZED]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+ 
+    try {
+      dataSet.recoverRbw(blocks[TEMPORARY], 
+          blocks[TEMPORARY].getGenerationStamp()+1, 
+          0L, blocks[TEMPORARY].getNumBytes());
+      Assert.fail("Should not have recovered a temporary replica " +
+          blocks[TEMPORARY]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.NON_RBW_REPLICA));
+    }
+
+    try {
+      dataSet.createRbw(blocks[TEMPORARY]);
+      Assert.fail("Should not have created a replica that had created as " +
+      		"temporary " + blocks[TEMPORARY]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+        
+    dataSet.recoverRbw(blocks[RBW], blocks[RBW].getGenerationStamp()+1, 
+        0L, blocks[RBW].getNumBytes());  // expect to be successful
+    
+    try {
+      dataSet.createRbw(blocks[RBW]);
+      Assert.fail("Should not have created a replica that had created as RBW " +
+          blocks[RBW]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    try {
+      dataSet.recoverRbw(blocks[RWR], blocks[RWR].getGenerationStamp()+1,
+          0L, blocks[RWR].getNumBytes());
+      Assert.fail("Should not have recovered a RWR replica " + blocks[RWR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.NON_RBW_REPLICA));
+    }
+
+    try {
+      dataSet.createRbw(blocks[RWR]);
+      Assert.fail("Should not have created a replica that was waiting to be " +
+      		"recovered " + blocks[RWR]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    try {
+      dataSet.recoverRbw(blocks[RUR], blocks[RUR].getGenerationStamp()+1,
+          0L, blocks[RUR].getNumBytes());
+      Assert.fail("Should not have recovered a RUR replica " + blocks[RUR]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          ReplicaNotFoundException.NON_RBW_REPLICA));
+    }
+
+    try {
+      dataSet.createRbw(blocks[RUR]);
+      Assert.fail("Should not have created a replica that was under recovery " +
+          blocks[RUR]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    try {
+      dataSet.recoverRbw(blocks[NON_EXISTENT],
+          blocks[NON_EXISTENT].getGenerationStamp()+1,
+          0L, blocks[NON_EXISTENT].getNumBytes());
+      Assert.fail("Cannot recover a non-existent replica " +
+          blocks[NON_EXISTENT]);
+    } catch (ReplicaNotFoundException e) {
+      Assert.assertTrue(
+          e.getMessage().contains(ReplicaNotFoundException.NON_EXISTENT_REPLICA));
+    }
+    
+    dataSet.createRbw(blocks[NON_EXISTENT]);
+  }
+  
+  private void testWriteToTemporary(FSDataset dataSet) throws IOException {
+    try {
+      dataSet.createTemporary(blocks[FINALIZED]);
+      Assert.fail("Should not have created a temporary replica that was " +
+      		"finalized " + blocks[FINALIZED]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+ 
+    try {
+      dataSet.createTemporary(blocks[TEMPORARY]);
+      Assert.fail("Should not have created a replica that had created as" +
+      		"temporary " + blocks[TEMPORARY]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    try {
+      dataSet.createTemporary(blocks[RBW]);
+      Assert.fail("Should not have created a replica that had created as RBW " +
+          blocks[RBW]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    try {
+      dataSet.createTemporary(blocks[RWR]);
+      Assert.fail("Should not have created a replica that was waiting to be " +
+      		"recovered " + blocks[RWR]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    try {
+      dataSet.createTemporary(blocks[RUR]);
+      Assert.fail("Should not have created a replica that was under recovery " +
+          blocks[RUR]);
+    } catch (ReplicaAlreadyExistsException e) {
+    }
+    
+    dataSet.createTemporary(blocks[NON_EXISTENT]);
+  }
+}

+ 25 - 14
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -519,7 +519,8 @@ public class NNThroughputBenchmark {
               .of(CreateFlag.OVERWRITE)), true, replication, BLOCK_SIZE);
               .of(CreateFlag.OVERWRITE)), true, replication, BLOCK_SIZE);
       long end = System.currentTimeMillis();
       long end = System.currentTimeMillis();
       for(boolean written = !closeUponCreate; !written; 
       for(boolean written = !closeUponCreate; !written; 
-        written = nameNode.complete(fileNames[daemonId][inputIdx], clientName));
+        written = nameNode.complete(fileNames[daemonId][inputIdx],
+                                    clientName, null));
       return end-start;
       return end-start;
     }
     }
 
 
@@ -685,8 +686,9 @@ public class NNThroughputBenchmark {
     
     
     NamespaceInfo nsInfo;
     NamespaceInfo nsInfo;
     DatanodeRegistration dnRegistration;
     DatanodeRegistration dnRegistration;
-    Block[] blocks;
+    ArrayList<Block> blocks;
     int nrBlocks; // actual number of blocks
     int nrBlocks; // actual number of blocks
+    long[] blockReportList;
 
 
     /**
     /**
      * Get data-node in the form 
      * Get data-node in the form 
@@ -705,7 +707,7 @@ public class NNThroughputBenchmark {
 
 
     TinyDatanode(int dnIdx, int blockCapacity) throws IOException {
     TinyDatanode(int dnIdx, int blockCapacity) throws IOException {
       dnRegistration = new DatanodeRegistration(getNodeName(dnIdx));
       dnRegistration = new DatanodeRegistration(getNodeName(dnIdx));
-      this.blocks = new Block[blockCapacity];
+      this.blocks = new ArrayList<Block>(blockCapacity);
       this.nrBlocks = 0;
       this.nrBlocks = 0;
     }
     }
 
 
@@ -738,19 +740,24 @@ public class NNThroughputBenchmark {
     }
     }
 
 
     boolean addBlock(Block blk) {
     boolean addBlock(Block blk) {
-      if(nrBlocks == blocks.length) {
-        LOG.debug("Cannot add block: datanode capacity = " + blocks.length);
+      if(nrBlocks == blocks.size()) {
+        LOG.debug("Cannot add block: datanode capacity = " + blocks.size());
         return false;
         return false;
       }
       }
-      blocks[nrBlocks] = blk;
+      blocks.set(nrBlocks, blk);
       nrBlocks++;
       nrBlocks++;
       return true;
       return true;
     }
     }
 
 
     void formBlockReport() {
     void formBlockReport() {
       // fill remaining slots with blocks that do not exist
       // fill remaining slots with blocks that do not exist
-      for(int idx = blocks.length-1; idx >= nrBlocks; idx--)
-        blocks[idx] = new Block(blocks.length - idx, 0, 0);
+      for(int idx = blocks.size()-1; idx >= nrBlocks; idx--)
+        blocks.set(idx, new Block(blocks.size() - idx, 0, 0));
+      blockReportList = new BlockListAsLongs(blocks,null).getBlockListAsLongs();
+    }
+
+    long[] getBlockReportList() {
+      return blockReportList;
     }
     }
 
 
     public int compareTo(String name) {
     public int compareTo(String name) {
@@ -760,6 +767,7 @@ public class NNThroughputBenchmark {
     /**
     /**
      * Send a heartbeat to the name-node and replicate blocks if requested.
      * Send a heartbeat to the name-node and replicate blocks if requested.
      */
      */
+    @SuppressWarnings("unused") // keep it for future blockReceived benchmark
     int replicateBlocks() throws IOException {
     int replicateBlocks() throws IOException {
       // register datanode
       // register datanode
       DatanodeCommand[] cmds = nameNode.sendHeartbeat(
       DatanodeCommand[] cmds = nameNode.sendHeartbeat(
@@ -889,8 +897,8 @@ public class NNThroughputBenchmark {
         nameNode.create(fileName, FsPermission.getDefault(), clientName,
         nameNode.create(fileName, FsPermission.getDefault(), clientName,
             new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.OVERWRITE)), true, replication,
             new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.OVERWRITE)), true, replication,
             BLOCK_SIZE);
             BLOCK_SIZE);
-        addBlocks(fileName, clientName);
-        nameNode.complete(fileName, clientName);
+        Block lastBlock = addBlocks(fileName, clientName);
+        nameNode.complete(fileName, clientName, lastBlock);
       }
       }
       // prepare block reports
       // prepare block reports
       for(int idx=0; idx < nrDatanodes; idx++) {
       for(int idx=0; idx < nrDatanodes; idx++) {
@@ -898,9 +906,12 @@ public class NNThroughputBenchmark {
       }
       }
     }
     }
 
 
-    private void addBlocks(String fileName, String clientName) throws IOException {
+    private Block addBlocks(String fileName, String clientName)
+    throws IOException {
+      Block prevBlock = null;
       for(int jdx = 0; jdx < blocksPerFile; jdx++) {
       for(int jdx = 0; jdx < blocksPerFile; jdx++) {
-        LocatedBlock loc = nameNode.addBlock(fileName, clientName);
+        LocatedBlock loc = nameNode.addBlock(fileName, clientName, prevBlock);
+        prevBlock = loc.getBlock();
         for(DatanodeInfo dnInfo : loc.getLocations()) {
         for(DatanodeInfo dnInfo : loc.getLocations()) {
           int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
           int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
           datanodes[dnIdx].addBlock(loc.getBlock());
           datanodes[dnIdx].addBlock(loc.getBlock());
@@ -910,6 +921,7 @@ public class NNThroughputBenchmark {
               new String[] {""});
               new String[] {""});
         }
         }
       }
       }
+      return prevBlock;
     }
     }
 
 
     /**
     /**
@@ -923,8 +935,7 @@ public class NNThroughputBenchmark {
       assert daemonId < numThreads : "Wrong daemonId.";
       assert daemonId < numThreads : "Wrong daemonId.";
       TinyDatanode dn = datanodes[daemonId];
       TinyDatanode dn = datanodes[daemonId];
       long start = System.currentTimeMillis();
       long start = System.currentTimeMillis();
-      nameNode.blockReport(dn.dnRegistration,
-          BlockListAsLongs.convertToArrayLongs(dn.blocks));
+      nameNode.blockReport(dn.dnRegistration, dn.getBlockReportList());
       long end = System.currentTimeMillis();
       long end = System.currentTimeMillis();
       return end-start;
       return end-start;
     }
     }

+ 176 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java

@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.TestFileCreation;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestBlockUnderConstruction {
+  static final String BASE_DIR = "/test/TestBlockUnderConstruction";
+  static final int BLOCK_SIZE = 8192; // same as TestFileCreation.blocksize
+  static final int NUM_BLOCKS = 5;  // number of blocks to write
+
+  private static MiniDFSCluster cluster;
+  private static DistributedFileSystem hdfs;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    cluster = new MiniDFSCluster(conf, 3, true, null);
+    cluster.waitActive();
+    hdfs = (DistributedFileSystem)cluster.getFileSystem();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if(hdfs != null) hdfs.close();
+    if(cluster != null) cluster.shutdown();
+  }
+
+  void writeFile(Path file, FSDataOutputStream stm, int size)
+  throws IOException {
+    long blocksBefore = stm.getPos() / BLOCK_SIZE;
+    
+    TestFileCreation.writeFile(stm, BLOCK_SIZE);
+    int blocksAfter = 0;
+    // wait until the block is allocated by DataStreamer
+    BlockLocation[] locatedBlocks;
+    while(blocksAfter <= blocksBefore) {
+      locatedBlocks = hdfs.getClient().getBlockLocations(
+          file.toString(), 0L, BLOCK_SIZE*NUM_BLOCKS);
+      blocksAfter = locatedBlocks == null ? 0 : locatedBlocks.length;
+    }
+  }
+
+  private void verifyFileBlocks(String file,
+                                boolean isFileOpen) throws IOException {
+    FSNamesystem ns = cluster.getNamesystem();
+    INodeFile inode = ns.dir.getFileINode(file);
+    assertTrue("File does not exist: " + inode.toString(), inode != null);
+    assertTrue("File " + inode.toString() +
+        " isUnderConstruction = " + inode.isUnderConstruction() +
+        " expected to be " + isFileOpen,
+        inode.isUnderConstruction() == isFileOpen);
+    BlockInfo[] blocks = inode.getBlocks();
+    assertTrue("File does not have blocks: " + inode.toString(),
+        blocks != null && blocks.length > 0);
+    
+    int idx = 0;
+    BlockInfo curBlock;
+    // all blocks but the last two should be regular blocks
+    for(; idx < blocks.length - 2; idx++) {
+      curBlock = blocks[idx];
+      assertTrue("Block is not complete: " + curBlock,
+          curBlock.isComplete());
+      assertTrue("Block is not in BlocksMap: " + curBlock,
+          ns.blockManager.getStoredBlock(curBlock) == curBlock);
+    }
+
+    // the penultimate block is either complete or
+    // committed if the file is not closed
+    if(idx > 0) {
+      curBlock = blocks[idx-1]; // penultimate block
+      assertTrue("Block " + curBlock +
+          " isUnderConstruction = " + inode.isUnderConstruction() +
+          " expected to be " + isFileOpen,
+          (isFileOpen && curBlock.isComplete()) ||
+          (!isFileOpen && !curBlock.isComplete() == 
+            (curBlock.getBlockUCState() ==
+              BlockUCState.COMMITTED)));
+      assertTrue("Block is not in BlocksMap: " + curBlock,
+          ns.blockManager.getStoredBlock(curBlock) == curBlock);
+    }
+
+    // the last block is under construction if the file is not closed
+    curBlock = blocks[idx]; // last block
+    assertEquals("Block " + curBlock +
+        " isComplete = " + curBlock.isComplete() +
+        " expected to be " + isFileOpen,
+        isFileOpen, !curBlock.isComplete());
+    assertTrue("Block is not in BlocksMap: " + curBlock,
+        ns.blockManager.getStoredBlock(curBlock) == curBlock);
+  }
+
+  @Test
+  public void testBlockCreation() throws IOException {
+    Path file1 = new Path(BASE_DIR, "file1.dat");
+    FSDataOutputStream out = TestFileCreation.createFile(hdfs, file1, 3);
+
+    for(int idx = 0; idx < NUM_BLOCKS; idx++) {
+      // write one block
+      writeFile(file1, out, BLOCK_SIZE);
+      // verify consistency
+      verifyFileBlocks(file1.toString(), true);
+    }
+
+    // close file
+    out.close();
+    // verify consistency
+    verifyFileBlocks(file1.toString(), false);
+  }
+
+  /**
+   * Test NameNode.getBlockLocations(..) on reading un-closed files.
+   */
+  @Test
+  public void testGetBlockLocations() throws IOException {
+    final NameNode namenode = cluster.getNameNode();
+    final Path p = new Path(BASE_DIR, "file2.dat");
+    final String src = p.toString();
+    final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 3);
+
+    // write a half block
+    int len = BLOCK_SIZE >>> 1;
+    writeFile(p, out, len);
+
+    for(int i = 1; i < NUM_BLOCKS; ) {
+      // verify consistency
+      final LocatedBlocks lb = namenode.getBlockLocations(src, 0, len);
+      final List<LocatedBlock> blocks = lb.getLocatedBlocks();
+      assertEquals(i, blocks.size());
+      final Block b = blocks.get(blocks.size() - 1).getBlock();
+      assertTrue(b instanceof BlockInfoUnderConstruction);
+
+      if (++i < NUM_BLOCKS) {
+        // write one more block
+        writeFile(p, out, BLOCK_SIZE);
+        len += BLOCK_SIZE;
+      }
+    }
+    // close file
+    out.close();
+  }
+}

+ 4 - 3
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -182,7 +182,8 @@ public class TestFsck extends TestCase {
       File baseDir = new File(System.getProperty("test.build.data",
       File baseDir = new File(System.getProperty("test.build.data",
                                                  "build/test/data"),"dfs/data");
                                                  "build/test/data"),"dfs/data");
       for (int i=0; i<8; i++) {
       for (int i=0; i<8; i++) {
-        File blockFile = new File(baseDir, "data" +(i+1)+ "/current/" + block);
+        File blockFile = new File(baseDir, "data" +(i+1) + 
+            MiniDFSCluster.FINALIZED_DIR_NAME + block);
         if(blockFile.exists()) {
         if(blockFile.exists()) {
           assertTrue(blockFile.delete());
           assertTrue(blockFile.delete());
         }
         }
@@ -294,8 +295,8 @@ public class TestFsck extends TestCase {
     File baseDir = new File(System.getProperty("test.build.data",
     File baseDir = new File(System.getProperty("test.build.data",
                                                "build/test/data"),"dfs/data");
                                                "build/test/data"),"dfs/data");
     for (int i=0; i < 6; i++) {
     for (int i=0; i < 6; i++) {
-      File blockFile = new File(baseDir, "data" + (i+1) + "/current/" +
-                                block);
+      File blockFile = new File(baseDir, "data" + (i+1) + 
+          MiniDFSCluster.FINALIZED_DIR_NAME + block);
       if (blockFile.exists()) {
       if (blockFile.exists()) {
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
         RandomAccessFile raFile = new RandomAccessFile(blockFile, "rw");
         FileChannel channel = raFile.getChannel();
         FileChannel channel = raFile.getChannel();

+ 2 - 1
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java

@@ -56,7 +56,8 @@ public class TestOverReplicatedBlocks extends TestCase {
       DataNodeProperties dnProps = cluster.stopDataNode(0);
       DataNodeProperties dnProps = cluster.stopDataNode(0);
       // remove block scanner log to trigger block scanning
       // remove block scanner log to trigger block scanning
       File scanLog = new File(System.getProperty("test.build.data"),
       File scanLog = new File(System.getProperty("test.build.data"),
-          "dfs/data/data1/current/dncp_block_verification.log.curr");
+          "dfs/data/data1" + MiniDFSCluster.FINALIZED_DIR_NAME + 
+          "dncp_block_verification.log.curr");
       //wait for one minute for deletion to succeed;
       //wait for one minute for deletion to succeed;
       for(int i=0; !scanLog.delete(); i++) {
       for(int i=0; !scanLog.delete(); i++) {
         assertTrue("Could not delete log file in one minute", i < 60);
         assertTrue("Could not delete log file in one minute", i < 60);

Some files were not shown because too many files changed in this diff