1
0
Prechádzať zdrojové kódy

HADOOP-1134. Add optimized checksum support to HDFS. Contributed by Raghu.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@556743 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 rokov pred
rodič
commit
712fc1d9c7
25 zmenil súbory, kde vykonal 3916 pridanie a 1006 odobranie
  1. 5 0
      CHANGES.txt
  2. 3 1
      src/java/org/apache/hadoop/dfs/Block.java
  3. 2092 0
      src/java/org/apache/hadoop/dfs/BlockCrcUpgrade.java
  4. 131 0
      src/java/org/apache/hadoop/dfs/ChecksumDistributedFileSystem.java
  5. 456 258
      src/java/org/apache/hadoop/dfs/DFSClient.java
  6. 234 0
      src/java/org/apache/hadoop/dfs/DataChecksum.java
  7. 475 354
      src/java/org/apache/hadoop/dfs/DataNode.java
  8. 15 2
      src/java/org/apache/hadoop/dfs/DatanodeProtocol.java
  9. 211 308
      src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
  10. 18 9
      src/java/org/apache/hadoop/dfs/FSConstants.java
  11. 52 11
      src/java/org/apache/hadoop/dfs/FSDataset.java
  12. 7 0
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  13. 116 3
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  14. 13 29
      src/java/org/apache/hadoop/dfs/JspHelper.java
  15. 5 0
      src/java/org/apache/hadoop/dfs/NameNode.java
  16. 16 26
      src/java/org/apache/hadoop/dfs/NamenodeFsck.java
  17. 2 0
      src/java/org/apache/hadoop/dfs/UpgradeObjectCollection.java
  18. 44 0
      src/java/org/apache/hadoop/fs/FileUtil.java
  19. 1 0
      src/java/org/apache/hadoop/fs/FsShell.java
  20. 12 4
      src/test/org/apache/hadoop/dfs/TestDFSShell.java
  21. 1 1
      src/test/org/apache/hadoop/dfs/TestDecommission.java
  22. 2 0
      src/test/org/apache/hadoop/dfs/TestFSInputChecker.java
  23. 2 0
      src/test/org/apache/hadoop/dfs/TestFSOutputSummer.java
  24. 1 0
      src/test/org/apache/hadoop/dfs/TestSmallBlock.java
  25. 2 0
      src/test/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java

+ 5 - 0
CHANGES.txt

@@ -362,6 +362,11 @@ Trunk (unreleased changes)
 113. HADOOP-1613.  Fix "DFS Health" page to display correct time of
      last contact.  (Dhruba Borthakur via cutting)
 
+114. HADOOP-1134.  Add optimized checksum support to HDFS.  Checksums
+     are now stored with each block, rather than as parallel files.
+     This reduces the namenode's memory requirements and increases
+     data integrity.  (Raghu Angadi via cutting)
+
 
 Release 0.13.0 - 2007-06-08
 

+ 3 - 1
src/java/org/apache/hadoop/dfs/Block.java

@@ -39,7 +39,9 @@ class Block implements Writable, Comparable {
   /**
    */
   public static boolean isBlockFilename(File f) {
-    if (f.getName().startsWith("blk_")) {
+    String name = f.getName();
+    if ( name.startsWith( "blk_" ) && 
+        name.indexOf( '.' ) < 0 ) {
       return true;
     } else {
       return false;

+ 2092 - 0
src/java/org/apache/hadoop/dfs/BlockCrcUpgrade.java

@@ -0,0 +1,2092 @@
+/**
+ * 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.dfs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.retry.*;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Daemon;
+
+import java.io.*;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.zip.CRC32;
+import java.util.concurrent.*;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.HashSet;
+import java.util.Random;
+
+/* This containtains information about CRC file and blocks created by
+ * ChecksumFileSystem for a given block.
+ */
+class BlockCrcInfo implements Writable {
+  
+  public static final int STATUS_DATA_BLOCK = 0;
+  public static final int STATUS_CRC_BLOCK = 1; // block belongs a .crc file
+  public static final int STATUS_NO_CRC_DATA = 2;
+  public static final int STATUS_UNKNOWN_BLOCK = 3;
+  public static final int STATUS_ERROR = 4; // unknown error
+  
+  int status = STATUS_ERROR;
+
+  String fileName = "";
+  long fileSize;
+  long startOffset;
+  long blockLen;
+  
+  LocatedBlock[] crcBlocks; // actual block locations.
+  int crcReplication; // expected replication.
+  
+  // set during 'offline upgrade'
+  boolean blockLocationsIncluded;
+  LocatedBlock blockLocations = new LocatedBlock(); 
+  
+  private void writeBlockLocations(LocatedBlock[] locArr, DataOutput out) 
+                                   throws IOException {
+    int len = ( locArr == null ) ? 0 : locArr.length;
+    out.writeInt( len );
+    if ( len > 0 ) {
+      for (LocatedBlock loc : locArr) {
+        loc.write( out );
+      }
+    }    
+  }
+  
+  private LocatedBlock[] readBlockLocations(DataInput in) throws IOException {
+    int len = in.readInt();
+    LocatedBlock[] locArr = (len > 0) ? new LocatedBlock[len] : null; 
+    for (int i=0; i<len; i++) {
+      locArr[i] = new LocatedBlock();
+      locArr[i].readFields( in );
+    }
+    return locArr;
+  }
+  
+  // Writable interface
+  
+  public void write(DataOutput out) throws IOException {
+    out.writeInt( status );
+    Text.writeString( out, fileName );
+    out.writeLong( fileSize );
+    out.writeLong( startOffset );
+    out.writeLong( blockLen );
+    writeBlockLocations(crcBlocks, out);
+    out.writeInt(crcReplication);
+    out.writeBoolean(blockLocationsIncluded);
+    blockLocations.write(out);
+  }
+  
+  public void readFields(DataInput in) throws IOException {
+    status = in.readInt();
+    fileName = Text.readString( in );
+    fileSize = in.readLong();
+    startOffset = in.readLong();
+    blockLen = in.readLong();
+    crcBlocks = readBlockLocations( in );
+    crcReplication = in.readInt();
+    blockLocationsIncluded = in.readBoolean();
+    blockLocations.readFields( in );
+  }
+}
+
+/**
+ * Consolidates various information regd upgrade of a single block at the
+ * Datanode.
+ */
+class DNBlockUpgradeInfo {
+  
+  Block block;
+  DataNode dataNode;
+  DatanodeProtocol namenode;
+  
+  BlockCrcInfo crcInfo; // CrcInfo fetched from the namenode.
+  boolean offlineUpgrade;
+  
+  /** Returns string that has block id and the associated file */
+  public String toString() {
+    return block + " (file: " +
+           ( (crcInfo == null || crcInfo.fileName == null) ? 
+             "Unknown" : crcInfo.fileName ) + ")";
+  }
+}
+
+/**
+ * This class contains various utilities for upgrade of DFS during switch
+ * to block level CRCs (HADOOP-1134).
+ */
+class BlockCrcUpgradeUtils {
+  
+  public static final Log LOG = 
+    LogFactory.getLog("org.apache.hadoop.dfs.BlockCrcUpgradeUtils");
+  
+  /* If some operation does not finish in this time, mostly something
+   * has gone wrong in the cluster (mostly it would be wrong configuration).
+   */ 
+  public static final int LONG_TIMEOUT_MINUTES = 5;
+  public static final int LONG_TIMEOUT_MILLISEC = LONG_TIMEOUT_MINUTES*60*1000;
+  public static final int PRE_BLOCK_CRC_LAYOUT_VERSION = -6;
+  
+  /** 
+   * Reads block data from a given set of datanodes. It tries each
+   * datanode in order and tries next datanode if fetch from prev datanode
+   * fails.
+   * 
+   * @param loc locations of the block
+   * @param offset offset into the block
+   * @param len len of the data to be read from the given offset
+   * @param buf buffer for data
+   * @param bufOffset offset into the buffer for writing
+   * @throws IOException
+   */ 
+  static void readFromRemoteNode(LocatedBlock loc, String crcFile,
+                                 long offset, long len,
+                                 byte[] buf, int bufOffset) 
+                                 throws IOException {
+    Block blk = loc.getBlock();
+    String dnStr = "";
+    
+    for (DatanodeInfo dn : loc.getLocations()) {
+      dnStr += dn.getName() + " ";
+      Socket dnSock = null;
+      try {
+        InetSocketAddress dnAddr = DataNode.createSocketAddr(dn.getName());
+        dnSock = new Socket();
+        dnSock.connect(dnAddr, FSConstants.READ_TIMEOUT);
+        dnSock.setSoTimeout(FSConstants.READ_TIMEOUT);
+        DFSClient.BlockReader reader = DFSClient.BlockReader.newBlockReader
+                    (dnSock, crcFile, blk.getBlockId(), offset, len, 
+                     (int)Math.min(len, 4096));
+        FileUtil.readFully(reader, buf, bufOffset, (int)len);
+        return;
+      } catch (IOException ioe) {
+        LOG.warn("Could not read " + blk + " from " + dn.getName() + " : " +
+                 StringUtils.stringifyException(ioe));
+      } finally {
+        if ( dnSock != null )
+          dnSock.close();
+      }
+    }
+    
+    throw new IOException("Could not fetch data for " + blk +
+                          " from datanodes " + dnStr);
+  }
+   
+  /**
+   * Reads data from an HDFS file from a given file-offset. 
+   * Before opening the file, it forces "fs.hdfs.impl" to
+   * "ChecksumDistributedFileSystem". So this is meant to read
+   * only files that have associcated ".crc" files.
+   * 
+   * @param filename HDFS complete filename for this file. 
+   * @param fileOffset fileOffset to read from
+   * @param len length of the data to read.
+   * @param namenodeAddr Namenode address for creating "hdfs://..." path.
+   * @param buf buffer to read into
+   * @param bufOffset offset into read buffer.
+   * @throws IOException
+   */
+  static void readDfsFileData(String filename, long fileOffset, int len, 
+                              InetSocketAddress namenodeAddr,
+                              byte[] buf, int bufOffset)
+                              throws IOException {
+    /*
+     * Read from an HDFS file.
+     */
+    Configuration conf = new Configuration();
+    //Makesure we use ChecksumFileSystem.
+    conf.set("fs.hdfs.impl",
+             "org.apache.hadoop.dfs.ChecksumDistributedFileSystem");
+    URI uri;
+    try {
+      uri = new URI("hdfs://" + namenodeAddr.getHostName() + ":" +
+                    namenodeAddr.getPort());
+    } catch (URISyntaxException e) {
+      throw new IOException("Got URISyntaxException for " + filename);
+    }
+    
+    FileSystem fs = FileSystem.get(uri, conf);
+    FSDataInputStream in = null;
+    try {
+      in = fs.open(new Path(filename));
+      if (fileOffset > 0) {
+        in.seek(fileOffset);
+      }
+      in.readFully(buf, bufOffset, len);
+    } finally {
+      FileUtil.closeStream(in);
+    }
+  }
+  
+  
+  /**
+   * This method is the main entry point for fetching CRC data for  
+   * a block from corresponding blocks for ".crc" file. 
+   * <br><br>
+   * 
+   * It first reads header from the ".crc" file and then invokes 
+   * readCrcBuf() to read the actual CRC data.
+   * It then writes the checksum to disk.
+   *
+   * 
+   * @param blockInfo
+   * @throws IOException when it fails to fetch CRC data for any reason.
+   */
+  static void readCrcFileData(DNBlockUpgradeInfo blockInfo)
+                              throws IOException {
+    
+    //First read the crc header  
+    byte[] header = new byte[8]; //'c r c \0 int '
+    int bytesRead = 0;
+    BlockCrcInfo crcInfo = blockInfo.crcInfo;
+    
+    for(int i=0; i<crcInfo.crcBlocks.length && 
+                 bytesRead < header.length; i++) {
+      
+      LocatedBlock loc = crcInfo.crcBlocks[i];
+      long toRead = Math.min(loc.getBlock().getNumBytes(),
+                             header.length-bytesRead);
+      readFromRemoteNode(loc, "."+crcInfo.fileName+".crc", 
+                         0, toRead, header, bytesRead);
+      bytesRead += toRead;
+    }
+    
+    if (bytesRead != header.length || header[0] != 'c' || 
+        header[1] != 'r' || header[2] != 'c' || header[3] != 0) {
+      // Should be very rare.
+      throw new IOException("Could not read header from crc file");
+    }
+    
+    int bytesPerChecksum = ((header[4] & 0xff) << 24) |
+                           ((header[5] & 0xff) << 16) |
+                           ((header[6] & 0xff) << 8) |
+                           (header[7] & 0xff);
+    
+    // sanity check the value. Is 100 MB good reasonable upper limt?
+    if (bytesPerChecksum < 1 || bytesPerChecksum > 100*1024*1024) {
+      throw new IOException("Insane value for bytesPerChecksum (" +
+                            bytesPerChecksum + ")");
+    }
+    
+    byte[] crcBuf = null;    
+        
+    try {
+      crcBuf = readCrcBuf(blockInfo, bytesPerChecksum);
+    } catch (IOException ioe) {
+      LOG.warn("Failed to fetch CRC data for " + blockInfo);
+      throw ioe;
+    } 
+    
+    writeCrcData(blockInfo, bytesPerChecksum, crcBuf);
+    /* After successful write(), we could inform the name node about it.
+     * or we can just inform name after all the blocks have been upgraded.
+     * Waiting for all the blocks to complete is probably better.
+     */
+  }
+  
+  /** 
+   * Low level function to create metadata file for a block with the
+   * CRC data. If crcBuf is null, it writes a metadata file with empty checksum.
+   * 
+   * @param blockInfo
+   * @param bytesPerChecksum 
+   * @param crcBuf buffer containing CRC. null implies metadata file should be
+   *        written with empty checksum.
+   * @throws IOException
+   */
+  static void writeCrcData(DNBlockUpgradeInfo blockInfo, int bytesPerChecksum,
+                           byte[] crcBuf) throws IOException {
+    Block block = blockInfo.block;
+    
+    File blockFile = blockInfo.dataNode.data.getBlockFile( block );
+    File metaFile = FSDataset.getMetaFile( blockFile );
+    
+    if ( bytesPerChecksum <= 0 ) {
+      if (crcBuf == null) {
+        bytesPerChecksum = blockInfo.dataNode.defaultBytesPerChecksum;
+      } else {
+        throw new IOException("Illegal Argument bytesPerChecksum(" +
+                              bytesPerChecksum);
+      }
+    }
+    
+    if ( metaFile.exists() ) {
+      if ( true ) {
+        throw new IOException("metadata file exists but unexpected for " +
+                              blockInfo);
+      }
+      // Verify the crcBuf. this should be removed.
+      if ( crcBuf == null )  {
+        return;
+      }
+      FileInputStream in = null;
+      try {
+        in = new FileInputStream( metaFile );
+        in.skip(7); //should be skipFully().
+        byte[] storedChecksum = new byte[ crcBuf.length ];
+        FileUtil.readFully(in, storedChecksum, 0, storedChecksum.length);
+        if ( !Arrays.equals(crcBuf, storedChecksum) ) {
+          throw new IOException("CRC does not match");
+        }
+      } finally {
+        FileUtil.closeStream(in);
+      }
+      return;
+    }
+    
+    File tmpBlockFile = null;
+    File tmpMetaFile = null;
+    DataOutputStream out = null;
+    try {
+      tmpBlockFile = blockInfo.dataNode.data.createTmpFile(null, block);
+      tmpMetaFile = FSDataset.getMetaFile( tmpBlockFile );
+      out = new DataOutputStream( new FileOutputStream(tmpMetaFile) );
+      
+      // write the header
+      out.writeShort( FSDataset.METADATA_VERSION );  
+      DataChecksum checksum = 
+        DataChecksum.newDataChecksum( ((crcBuf == null) ?
+                                       DataChecksum.CHECKSUM_NULL :
+                                       DataChecksum.CHECKSUM_CRC32 ),
+                                       bytesPerChecksum );
+      checksum.writeHeader(out);
+      if (crcBuf != null) {
+        out.write(crcBuf);
+      }
+      out.close();
+      out = null;
+      
+      if ( !tmpMetaFile.renameTo( metaFile ) ) {
+        throw new IOException("Could not rename " + tmpMetaFile + " to " +
+                              metaFile);
+      }
+    } finally {
+      FileUtil.closeStream(out);
+      if ( tmpBlockFile != null ) {
+        tmpBlockFile.delete();
+      }
+    }
+  }
+  
+  /**
+   * This regenerates CRC for a block by checksumming few bytes at either end 
+   * of the block (read from the HDFS file) and the local block data. This is 
+   * invoked only when either end of the block  doesn't fall on checksum
+   * boundary (very uncommon). <i>bytesBefore</i> or <i>bytesAfter</i>
+   * should have non-zero length.
+   *  
+   * @param blockInfo
+   * @param oldCrcBuf CRC buffer fetched from ".crc" file
+   * @param bytesPerChecksum
+   * @param bytesBefore bytes located before this block in the file
+   * @param bytesAfter bytes located after this block in the file
+   * @return Returns the generated CRC in a buffer.
+   * @throws IOException
+   */
+  static byte[] regenerateCrcBuf(DNBlockUpgradeInfo blockInfo,
+                                 byte[] oldCrcBuf, int bytesPerChecksum,
+                                 byte[] bytesBefore, byte[] bytesAfter)
+                                 throws IOException {
+    
+    DataChecksum verificationChecksum = DataChecksum.newDataChecksum
+                                         (DataChecksum.CHECKSUM_CRC32, 
+                                          bytesPerChecksum);
+    DataChecksum newChecksum = DataChecksum.newDataChecksum
+                                         (DataChecksum.CHECKSUM_CRC32, 
+                                          bytesPerChecksum);
+    
+    BlockCrcInfo crcInfo = blockInfo.crcInfo;
+    
+    int checksumSize = newChecksum.getChecksumSize();
+    int newCrcSize = (int) (crcInfo.blockLen/bytesPerChecksum*checksumSize);
+    if ( crcInfo.blockLen%bytesPerChecksum > 0 ) {
+      newCrcSize += checksumSize;
+    }
+    byte[] newCrcBuf = new byte[newCrcSize];
+    int newCrcOffset = 0;
+    int oldCrcOffset = 0;
+    
+    Block block = blockInfo.block;
+    File blockFile = blockInfo.dataNode.data.getBlockFile( block );
+    if ( blockFile == null || !blockFile.exists() ) {
+      throw new IOException("Block file "  + 
+                            ((blockFile != null) ? blockFile.getAbsolutePath()
+                              : "NULL") + " does not exist.");
+    }
+    
+    byte[] blockBuf = new byte[bytesPerChecksum];
+    FileInputStream in = null;
+       
+    try {
+      boolean chunkTainted = false;
+      boolean prevChunkTainted = false;
+      long bytesRead = 0;
+
+      if ( bytesBefore.length > 0 ) {
+        verificationChecksum.update(bytesBefore, 0, bytesBefore.length);
+      }
+      int verifyLen = bytesPerChecksum - bytesBefore.length;
+      long verifiedOffset = -bytesBefore.length;
+      long writtenOffset = 0;
+
+      in = new FileInputStream( blockFile );
+      
+      while ( bytesRead <= crcInfo.blockLen ) {        
+        /* In each iteration we read number of bytes required for newChecksum,
+         * except in the last iteration where we read 0 bytes.
+         * newChecksum updated in an iteration is written in the next 
+         * iteration, because only in the next iteration will we be 
+         * gauranteed that we have verified enough data to write newChecksum.
+         * All the comparisions below are chosen to enforce the above.
+         */
+        int toRead = (int) Math.min(crcInfo.blockLen - bytesRead, 
+                                    bytesPerChecksum);
+        /* if bytesBefore.length == 0, then we need not rechecksum but
+         * simply copy from oldCrcBuf for most of the block.
+         * But we are not optimizing for this case.
+         */
+        if ( toRead > 0 ) {
+          FileUtil.readFully(in, blockBuf, 0, toRead);
+        }
+
+        if ( (toRead == 0 && bytesAfter.length > 0) || toRead >= verifyLen ) {
+          if ( toRead > 0 ) {
+            verificationChecksum.update(blockBuf, 0, verifyLen);
+          }
+          prevChunkTainted = chunkTainted;
+          chunkTainted = !verificationChecksum.compare(oldCrcBuf, oldCrcOffset);
+          oldCrcOffset += checksumSize;
+          verifiedOffset += bytesPerChecksum;
+          verificationChecksum.reset();
+        }
+
+        /* We update newCrcBuf only after all the bytes checksummed are
+         * verified.
+         */
+        long diff = verifiedOffset - writtenOffset;
+        if ( toRead == 0 || diff > bytesPerChecksum || 
+            bytesRead >= verifiedOffset ) {
+          // decide if we need to reset the checksum.
+          if ( (diff > bytesPerChecksum && prevChunkTainted) || 
+              (diff < 2L*bytesPerChecksum && chunkTainted) ) {
+            LOG.warn("Resetting checksum for " + blockInfo + " at offset "
+                     + writtenOffset);
+            newChecksum.reset();
+          }
+          newChecksum.writeValue(newCrcBuf, newCrcOffset, true);
+          newCrcOffset += checksumSize;
+          writtenOffset += Math.min( crcInfo.blockLen-writtenOffset,
+                                     bytesPerChecksum );
+        }
+
+        if ( toRead == 0 ) {
+          //Finally done with update the new CRC buffer!
+          break;
+        }
+
+        if ( toRead != verifyLen ) {
+          int tmpOff = ( toRead > verifyLen ) ? verifyLen : 0; 
+          verificationChecksum.update(blockBuf, tmpOff, toRead-tmpOff);
+        }
+
+        bytesRead += toRead;
+        if ( bytesRead == crcInfo.blockLen && bytesAfter.length > 0 ) {
+          /* We are at the edge.
+           * if bytesBefore.length == 0, then blockLen % bytesPerChecksum
+           * can not be 0.
+           */
+          verificationChecksum.update(bytesAfter, 0, bytesAfter.length);
+        }
+        newChecksum.update(blockBuf, 0, toRead);
+      }
+
+      //XXX Remove the assert.
+      assert newCrcBuf.length == newCrcOffset : "something is wrong"; 
+      return newCrcBuf;
+    } finally {
+      FileUtil.closeStream(in);
+    }
+  }
+
+  /**
+   * Reads multiple copies of CRC data from different replicas and  
+   * compares them. It selects the CRC data that matches on majority
+   * of the copies. If first 'replication/2+1' copies match with each
+   * other it does not read rest of the copies of CRC data, this would be
+   * the common case for most the blocks.
+   * <br><br>
+   * 
+   * When CRC data falls on multiple blocks, the permutations it tries are 
+   * pretty simple. If there 2 blocks involved, with replication of 3, it tries
+   * 3 combinations and not 9. Currently it does not read more replicas
+   * if CRC data is over-replicated, reads only up to file's 'replication'.
+   * <br><br> 
+   * 
+   * If all copies are different, it returns the first one.
+   * <br>
+   * 
+   * @param blockInfo
+   * @param crcStart start offset into the ".crc" file
+   * @param crcSize length of the CRC data
+   * @return Returns buffer containing crc data. buffer.length == crcSize
+   * @throws IOException when it can not fetch even one copy of the CRC data.
+   */
+  static byte[] findMajorityCrcBuf(DNBlockUpgradeInfo blockInfo,
+                                   long crcStart, int crcSize)
+                                   throws IOException {
+    
+    Block block = blockInfo.block;
+    BlockCrcInfo crcInfo = blockInfo.crcInfo;
+    int replication = crcInfo.crcReplication;
+    
+    // internal/local class
+    class CrcBufInfo {
+      CrcBufInfo(byte[] b, long checksum) {
+        buf = b;
+        crc32 = checksum;
+      }
+      byte[] buf;
+      long crc32;
+      int  numMatches = 0;
+    }
+    
+    CrcBufInfo[] bufInfoArr = new CrcBufInfo[replication];
+    int numBufs = 0;
+    CRC32 crc32 = new CRC32();
+    boolean atleastOneNewReplica = true;
+    
+    for(int i=0; i<replication && atleastOneNewReplica; i++) {
+      /* when crc data falls on multiple blocks, we don't
+       * try fetching from all combinations of the replicas. We could.
+       * We try only crcReplication combinations.
+       */
+      int bytesRead = 0;
+      byte[] buf = new byte[crcSize];
+      long offset = 0;
+      atleastOneNewReplica = false;
+      String crcFileName = "."+blockInfo.crcInfo.fileName+".crc";
+      for (LocatedBlock loc : crcInfo.crcBlocks) {          
+
+        long blockStart = crcStart - offset + bytesRead; 
+        long blockSize = loc.getBlock().getNumBytes();
+        offset += blockSize;
+        if ( blockSize <= blockStart ){
+          continue;
+        }
+        
+        DatanodeInfo dn;
+        DatanodeInfo [] dnArr = loc.getLocations();
+        if ( dnArr.length > i ) {
+          dn = dnArr[i];
+          atleastOneNewReplica = true;
+        } else {
+          // if all the data is in single block, then no need to read
+          if ( (bytesRead == 0 || !atleastOneNewReplica) &&
+               (blockSize - blockStart) >= (crcSize - bytesRead) ) {
+            break;
+          }
+          dn = dnArr[dnArr.length-1];
+        }
+        
+        try {
+          DatanodeInfo[] tmpArr = new DatanodeInfo[1];
+          tmpArr[0] = dn;
+          long toRead = Math.min(crcSize - bytesRead, blockSize-blockStart);
+          readFromRemoteNode(new LocatedBlock(loc.getBlock(), tmpArr),
+                             crcFileName, blockStart, toRead, buf, bytesRead);
+          bytesRead += toRead;
+        } catch (IOException ioe) {
+          LOG.warn("Error while fetching crc data from " + dn.getName() +
+                   "for " + blockInfo + StringUtils.stringifyException(ioe));
+          break;
+        }
+        
+        if ( bytesRead < crcSize ) {
+          continue;
+        }
+        
+        crc32.reset();
+        crc32.update(buf, 0, crcSize);
+        long crc = crc32.getValue();
+
+        for(int j=0; j<numBufs+1; j++) {
+          if ( j < numBufs && crc != bufInfoArr[j].crc32 ) {
+            LOG.warn("Mismatch in crc for " + blockInfo);
+            continue;
+          }
+
+          CrcBufInfo info = ( j < numBufs ) ? bufInfoArr[j] :
+            new CrcBufInfo(buf, crc);
+          
+          info.numMatches++;
+          if (info.numMatches >= (replication/2 + replication%2)) {
+            LOG.info("readCrcBuf: " + info.numMatches + 
+                     " out of " + replication + " matched for " + blockInfo);
+            return buf;
+          }
+
+          if ( j == numBufs ) {
+            bufInfoArr[ numBufs++ ] = info;
+          }
+          break;
+        }
+        
+        // Done reading crcSize bytes.
+        break;
+      }
+    }
+    
+    /* Now we have an error or some buffer that might not have 
+     * absolute majority.
+     * Try to pick the buffer that that has max number of matches.
+     */
+    CrcBufInfo selectedBuf = null;
+    for (int i=0; i<numBufs; i++) {
+      CrcBufInfo info = bufInfoArr[i];
+      if (selectedBuf == null || selectedBuf.numMatches < info.numMatches) {
+        selectedBuf = info;
+      }
+    }
+
+    if (selectedBuf == null) {      
+      throw new IOException("Could not fetch any crc data for " + block);
+    }
+
+    LOG.info("readCrcBuf: " + selectedBuf.numMatches + 
+             " out of " + replication + " matched for " + blockInfo);
+    
+    //Print a warning if numMatches is 1?
+    return  selectedBuf.buf;      
+  }
+  
+  /**
+   * Reads CRC data for a block from corresponding ".crc". Usually it 
+   * fetches CRC from blocks that belong to ".crc" file. When the given 
+   * block does not start or end on a checksum boundary, it would read data
+   * from ".crc" DFS itself (as opposed to directly fetching from blocks) and
+   * regenerate CRC for the block.
+   * 
+   * @param blockInfo
+   * @param bytesPerChecksum bytesPerChecksum from ".crc" file header 
+   * @return buffer containing CRC data for the block.
+   * @throws IOException
+   */
+  static byte[] readCrcBuf(DNBlockUpgradeInfo blockInfo, int bytesPerChecksum)
+                           throws IOException {
+    
+    BlockCrcInfo crcInfo = blockInfo.crcInfo;
+    
+    int checksumSize = 4; // CRC32.
+    
+    /* Following two arrayas are used in the case where block 
+     * does not fall on edges. This happens when 'dfs.block.size' is not
+     * a multiple of 'io.bytes.per.checksum'. Apparently it can also happen
+     * because of a known bug in DFSClient, but I am not sure. 
+     */
+    byte [] bytesBefore = new byte[(int)(crcInfo.startOffset % 
+                                         bytesPerChecksum)]; 
+    int tmpMod = (int) ((crcInfo.blockLen + bytesBefore.length) % 
+                        bytesPerChecksum);
+    if ( tmpMod != 0 ) {
+      tmpMod = (int) Math.min(bytesPerChecksum - tmpMod,
+                              (crcInfo.fileSize - crcInfo.startOffset - 
+                               crcInfo.blockLen));
+    }
+    byte [] bytesAfter = new byte[tmpMod];
+    
+    if ( bytesBefore.length > 0 || bytesAfter.length > 0 ) {
+      if ( bytesBefore.length > 0 ) {
+        readDfsFileData(crcInfo.fileName,
+                        crcInfo.startOffset-bytesBefore.length,
+                        bytesBefore.length,
+                        blockInfo.dataNode.getNameNodeAddr(),
+                        bytesBefore, 0);
+      }
+      if ( bytesAfter.length > 0 ) {
+        readDfsFileData(crcInfo.fileName,  
+                        crcInfo.startOffset+crcInfo.blockLen, 
+                        bytesAfter.length,
+                        blockInfo.dataNode.getNameNodeAddr(),                        
+                        bytesAfter, 0);
+      }
+    }
+    
+    // Now fetch the crc. XXX change 8 to HEADER_SIZE.
+    long crcStart = 8 + ( (crcInfo.startOffset-bytesBefore.length)/
+                          bytesPerChecksum * checksumSize );
+    
+    long tmpLen = crcInfo.blockLen + bytesBefore.length + bytesAfter.length;
+    int crcSize = (int) (tmpLen/bytesPerChecksum*checksumSize);
+    if (tmpLen % bytesPerChecksum > 0) {
+      crcSize += checksumSize;
+    }
+    
+    byte[] crcBuf = findMajorityCrcBuf(blockInfo, crcStart, crcSize);
+    
+    if ( bytesBefore.length > 0 || bytesAfter.length > 0 ) {
+      /* We have crc for data that is larger than the blocks data.
+       * So regenerate new CRC data.
+       */
+      crcBuf = regenerateCrcBuf(blockInfo, crcBuf, 
+                                bytesPerChecksum, bytesBefore, bytesAfter);
+    }
+    return crcBuf;
+  }
+  
+  /** Generates CRC data for a block by reading block data from the local
+   * storage. Usually this is used as the last resort option to generate
+   * metadata for a block.
+   * <br><br>
+   * 
+   * Bytes per checksum is based on config (io.bytes.per.checksum).
+   * 
+   * @param blockInfo Block information.
+   * @throws IOException
+   */
+  static void generateLocalCrcData( DNBlockUpgradeInfo blockInfo ) 
+                                   throws IOException {
+    
+    Block block = blockInfo.block;
+    File blockFile = blockInfo.dataNode.data.getBlockFile( block );
+    if (blockFile == null || !blockFile.exists()) {
+      throw new IOException("Could not local file for block");
+    }
+    long blockLen = blockFile.length();
+    if ( blockLen != blockFile.length()) {
+      LOG.warn("Mismatch in length for block: local file size is " +
+               blockLen + " but should be " + blockInfo.crcInfo.blockLen +
+               " for " + blockInfo + ". Using local file size for CRC");
+    }
+    int bytesPerChecksum = blockInfo.dataNode.defaultBytesPerChecksum;
+    DataChecksum checksum = 
+      DataChecksum.newDataChecksum(DataChecksum.CHECKSUM_CRC32,
+                                   bytesPerChecksum);
+    int checksumSize = checksum.getChecksumSize();
+    int crcBufSize = (int) ((blockLen/bytesPerChecksum)*checksumSize);
+    if (blockLen%bytesPerChecksum != 0) {
+      crcBufSize += checksumSize;
+    }
+    byte crcBuf[] = new byte[crcBufSize];
+
+    
+    InputStream in = null;
+    try {
+      in = new FileInputStream(blockFile);
+      
+      long totalRead = 0;
+      byte[] dataBuf = new byte[bytesPerChecksum];
+      int crcBufPos = 0;     
+      
+      while (totalRead < blockLen) {
+        int toRead = Math.min((int)(blockLen - totalRead), bytesPerChecksum);
+        FileUtil.readFully(in, dataBuf, 0, toRead );
+        
+        checksum.update(dataBuf, 0, toRead);
+        crcBufPos += checksum.writeValue(crcBuf, crcBufPos, true);
+        checksum.reset();
+        
+        totalRead += toRead;
+      }
+    } finally {
+      FileUtil.closeStream(in);
+    }
+    
+    writeCrcData(blockInfo, bytesPerChecksum, crcBuf);
+  }
+    
+  /**
+   * Fetches CRC data from a remote node. Sends READ_BLOCK_METADATA
+   * command. Extracts CRC information and returns it.
+   */
+  static byte[] readCrcFromReplica(DNBlockUpgradeInfo blockInfo,
+                                   DatanodeInfo dnInfo,
+                                   DataChecksum[] checksumArr
+                                   ) throws IOException {
+    Socket dnSock = null;
+    
+    String errMsg = "";
+    
+    try {
+      do {
+        InetSocketAddress dnAddr = DataNode.createSocketAddr(dnInfo.getName());
+        dnSock = new Socket();
+        dnSock.connect(dnAddr, FSConstants.READ_TIMEOUT);
+        dnSock.setSoTimeout(FSConstants.READ_TIMEOUT);
+
+        DataOutputStream out = new DataOutputStream(dnSock.getOutputStream());
+        DataInputStream in = new DataInputStream(dnSock.getInputStream());
+
+        // Write the header:
+        out.writeShort( DataNode.DATA_TRANFER_VERSION );
+        out.writeByte( DataNode.OP_READ_METADATA );
+        out.writeLong( blockInfo.block.getBlockId() );
+
+        byte reply = in.readByte();
+        if ( reply != DataNode.OP_STATUS_SUCCESS ) {
+          errMsg = "Got error(" + reply + ") in reply";
+          break;
+        }
+
+        // There is no checksum for this transfer.
+        int len = in.readInt();
+        int headerLen = 2 + DataChecksum.HEADER_LEN;
+        
+        if ( len < headerLen ) {
+          errMsg = "len is too short";
+          break;
+        }
+        
+        if ( len > 0 ) {
+          // Verify that version is same
+          short version = in.readShort();
+          if ( version != FSDataset.METADATA_VERSION ) {
+            errMsg = "Version mismatch";
+            break;
+          }
+
+          DataChecksum checksum = DataChecksum.newDataChecksum(in);
+          
+          if ( checksum.getChecksumType() != DataChecksum.CHECKSUM_CRC32 ) {
+            errMsg = "Checksum is not CRC32";
+            break;
+          }
+          
+          int crcBufLen = (int) (( (blockInfo.crcInfo.blockLen + 
+                                    checksum.getBytesPerChecksum()-1)/
+                                    checksum.getBytesPerChecksum() ) *
+                                  checksum.getChecksumSize());
+          
+          if ( (len - headerLen) != crcBufLen ) {
+            errMsg = "CRC data is too short";
+          }
+          byte[] crcBuf = new byte[crcBufLen];
+          in.readFully(crcBuf);
+          
+          //just read the last int
+          in.readInt();
+          
+          checksumArr[0] = checksum;
+          return crcBuf;
+        }
+      } while (false);
+    } finally {
+      FileUtil.closeSocket( dnSock );
+    }
+    
+    throw new IOException("Error while fetching CRC from replica on " +
+                          dnInfo.getName() + ": " + errMsg); 
+  }
+    
+    
+  /**
+   * Reads metadata from the replicas and writes the CRC from the
+   * first successful fetch.
+   */
+  static void readCrcFromReplicas(DNBlockUpgradeInfo blockInfo) 
+                                  throws IOException {
+    
+    /* Reads metadata from from the replicas */
+    DatanodeInfo[] dnArr = ( blockInfo.crcInfo.blockLocationsIncluded ? 
+                             blockInfo.crcInfo.blockLocations.getLocations() :
+                            new DatanodeInfo[0]);
+    
+    DataChecksum[] checksumArr = new DataChecksum[1];
+    IOException ioe = null;    
+    String myName = blockInfo.dataNode.dnRegistration.getName();
+    
+    for (DatanodeInfo dnInfo : dnArr) {
+      if ( dnInfo.getName().equals(myName) ) {
+        LOG.info("XXX skipping crcInfo fetch from " + dnInfo.getName());
+      } else {
+        try {
+          byte[] crcBuf = readCrcFromReplica(blockInfo, dnInfo, checksumArr);
+          LOG.info("read crcBuf from " + dnInfo.getName() + " for " +
+                   blockInfo);
+          
+          writeCrcData(blockInfo, checksumArr[0].getBytesPerChecksum(), 
+                       crcBuf);
+          return;
+        } catch (IOException e) {
+          LOG.warn("Could not fetch crc data from " + dnInfo.getName() +
+                   " : " + e);
+          ioe = e;
+        }
+      }
+    }
+    
+    if ( ioe != null ) {
+      throw ioe;
+    }
+    
+    throw new IOException("Could not fetch crc data from any node");
+  }
+  
+  
+  /**
+   * The method run by the upgrade threads. It contacts namenode for 
+   * information about the block and invokes appropriate method to create 
+   * metadata file for the block.
+   * 
+   * @param blockInfo
+   */
+  static void upgradeBlock( DNBlockUpgradeInfo blockInfo ) {
+
+    UpgradeCommand ret = null;
+    
+    if ( blockInfo.offlineUpgrade ) {
+      blockInfo.crcInfo = getBlockLocations(blockInfo.namenode, 
+                                            blockInfo.block);
+    } else { 
+      ret = sendCommand(blockInfo.namenode, new CrcInfoCommand(blockInfo.block),
+                        -1);
+    }
+    
+    /* Should be removed.
+    if (true) {
+      int sleepTime = (new Random(blockInfo.block.getBlockId())).nextInt(10);
+       BlockCrcUpgradeUtils.sleep(sleepTime,"XXX before upgrading the block");
+    } */
+    
+    try {
+     
+      if ( !blockInfo.offlineUpgrade ) {
+        if ( ret == noUpgradeOnNamenode ) {
+          throw new IOException("No upgrade is running on Namenode");
+        }
+
+        if ( ret == null || ((CrcInfoCommandReply)ret).crcInfo == null ) {
+          throw new IOException("Could not get crcInfo from Namenode");
+        }
+
+        blockInfo.crcInfo = ((CrcInfoCommandReply)ret).crcInfo;
+      }
+      
+      if ( blockInfo.crcInfo == null ) {
+        throw new IOException("Could not fetch crcInfo for " + 
+                              blockInfo.block);
+      }
+      
+      switch (blockInfo.crcInfo.status) {
+      
+      case BlockCrcInfo.STATUS_DATA_BLOCK:
+        try {
+          if (blockInfo.offlineUpgrade) {
+            readCrcFromReplicas(blockInfo);
+          } else {
+            readCrcFileData(blockInfo);
+          }
+        } catch (IOException e) {
+          LOG.warn("Exception in " + 
+                   ((blockInfo.offlineUpgrade) ? 
+                    "readCrcFromReplicas()" : "readCrcFileData()") + 
+                   " for " + blockInfo + ". will try to generate local crc.");
+          throw e;
+        }
+        break;
+        
+      case BlockCrcInfo.STATUS_NO_CRC_DATA:
+        generateLocalCrcData(blockInfo);
+        break;
+        
+      case BlockCrcInfo.STATUS_UNKNOWN_BLOCK:
+        LOG.info("block is already deleted. Will create an empty " +
+                 "metadata file for " + blockInfo);
+        writeCrcData(blockInfo, 0, null);
+        break;
+        
+      case BlockCrcInfo.STATUS_CRC_BLOCK :
+        writeCrcData(blockInfo, 0, null);
+        break;
+        
+      case BlockCrcInfo.STATUS_ERROR:
+        LOG.info("unknown error. will generate local crc data for " + 
+                 blockInfo);
+        generateLocalCrcData(blockInfo);
+        break;
+        
+      default:
+        LOG.error("Unknown status from Namenode for " + blockInfo);
+        assert false : "Unknown status from Namenode";
+      }
+    } catch (IOException e) {
+      LOG.warn("Could not fetch crc for " + blockInfo + 
+               " will generate local crc data : exception :" + 
+               StringUtils.stringifyException(e));
+      try {
+        // last option:
+        generateLocalCrcData(blockInfo);
+      } catch (IOException ioe) {
+        LOG.warn("Could not generate local crc data for " + blockInfo +
+                 " : exception : " + StringUtils.stringifyException(ioe));
+      }
+    }
+  }
+
+  static UpgradeCommand noUpgradeOnNamenode = new UpgradeCommand();
+  
+  /** 
+   * retries command in case of timeouts. <br>
+   * If retries is < 0, retries forever. <br>
+   * NOTE: Does not throw an exception. <br>
+   */
+  static UpgradeCommand sendCommand(DatanodeProtocol namenode,
+                                    UpgradeCommand cmd, int retries) {
+    for(int i=0; i<=retries || retries<0; i++) {
+      try {
+        UpgradeCommand reply = namenode.processUpgradeCommand(cmd);
+        if ( reply == null ) {
+          /* namenode might not be running upgrade or finished
+           * an upgrade. We just return a static object */
+          return noUpgradeOnNamenode;
+        }
+        return reply;
+      } catch (IOException e) {
+        // print the stack trace only for the last retry.
+        LOG.warn("Exception while sending command " + 
+                 cmd.getAction() + ": " + e +
+                 ((retries<0 || i>=retries)? "... will retry ..." : 
+                   ": " + StringUtils.stringifyException(e)));
+      }
+    }
+    return null; 
+  }
+  
+  // Similar to sendCommand(). Invokes command in a loop.
+  static BlockCrcInfo getBlockLocations(DatanodeProtocol namenode, 
+                                        Block block) {
+    for (;;) {
+      try {
+        return namenode.blockCrcUpgradeGetBlockLocations(block);
+      } catch (IOException e) {
+        LOG.warn("Exception while fetching block Locations from namenode: " +
+                 e + " ... will retry ...");
+      }
+    }
+  }
+  
+    
+  /** sleep method that catches and swallows InterruptedException
+   */
+  static void sleep(int seconds, String message) {
+    if ( message != null ) {
+      LOG.info("XXX Sleeping for " + seconds + " seconds. msg: " + message);
+    }
+    try {
+      Thread.sleep(seconds*1000L);
+    } catch (InterruptedException ignored) {}
+  }
+  
+  /* Upgrade commands */
+  static final int DN_CMD_STATS = 200;
+  static final int DN_CMD_CRC_INFO = 201;
+  static final int DN_CMD_CRC_INFO_REPLY = 202;
+  
+  // what is this version for?
+  static final int DN_CMD_VERSION = PRE_BLOCK_CRC_LAYOUT_VERSION;
+
+  static class DatanodeStatsCommand extends UpgradeCommand {
+    DatanodeID datanodeId;
+    int blocksUpgraded;
+    int blocksRemaining;
+    int errors;
+    
+    DatanodeStatsCommand() {
+      super(DN_CMD_STATS, DN_CMD_VERSION, (short)0);
+      datanodeId = new DatanodeID();
+    }
+    
+    public DatanodeStatsCommand(short status, DatanodeID dn,
+                                int blocksUpgraded, int blocksRemaining,
+                                int errors) {
+      super(DN_CMD_STATS, DN_CMD_VERSION, status);
+      
+      //copy so that only ID part gets serialized
+      datanodeId = new DatanodeID(dn); 
+      this.blocksUpgraded = blocksUpgraded;
+      this.blocksRemaining = blocksRemaining;
+      this.errors = errors;
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      datanodeId.readFields(in);
+      blocksUpgraded = in.readInt();
+      blocksRemaining = in.readInt();
+      errors = in.readInt();
+    }
+
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      datanodeId.write(out);
+      out.writeInt(blocksUpgraded);
+      out.writeInt(blocksRemaining);
+      out.writeInt(errors);
+    }
+  }
+
+  static class CrcInfoCommand extends UpgradeCommand {
+    Block block;
+    
+    public CrcInfoCommand() {
+      super(DN_CMD_CRC_INFO, DN_CMD_VERSION, (short)0);      
+      block = new Block();
+    }
+    
+    public CrcInfoCommand(Block blk) {
+      // We don't need status
+      super(DN_CMD_CRC_INFO, DN_CMD_VERSION, (short)0);
+      block = blk;
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      block.readFields(in);
+    }
+
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      block.write(out);
+    }
+  }
+  
+  static class CrcInfoCommandReply extends UpgradeCommand {
+    BlockCrcInfo crcInfo;
+    
+    public CrcInfoCommandReply(){
+      super(DN_CMD_CRC_INFO_REPLY, DN_CMD_VERSION, (short)0);      
+      crcInfo = new BlockCrcInfo();
+    }
+    
+    public CrcInfoCommandReply(BlockCrcInfo info) {
+      super(DN_CMD_CRC_INFO_REPLY, DN_CMD_VERSION, (short)0);
+      crcInfo = info;
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      crcInfo.readFields(in);
+    }
+
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      crcInfo.write(out);
+    }
+  }
+
+  static class BlockCrcUpgradeStatusReport extends UpgradeStatusReport {
+
+    String extraText = "";
+    
+    public BlockCrcUpgradeStatusReport() {
+    }
+    
+    public BlockCrcUpgradeStatusReport(int version, short status,
+                                       String extraText) {
+      super(version, status);
+      this.extraText = extraText;
+    }
+    
+    @Override
+    public String getStatusText(boolean details) {
+      return super.getStatusText(details) + "\n\n" + extraText;
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      super.readFields(in);
+      extraText = Text.readString(in);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      super.write(out);
+      Text.writeString(out, extraText);
+    }
+  }
+}
+
+/**
+ * This class checks all the blocks and upgrades any that do not have 
+ * meta data associated with them.
+ * 
+ * Things to consider : 
+ *   Should it rescan at the end of the loop?
+ */
+class BlockCrcUpgradeObjectDatanode extends UpgradeObjectDatanode {
+
+  public static final Log LOG = 
+    LogFactory.getLog("org.apache.hadoop.dfs.BlockCrcUpgradeObjectDatanode");
+
+  DatanodeProtocol namenode;
+  
+  // stats
+  int blocksPreviouslyUpgraded;
+  int blocksToUpgrade;
+  int blocksUpgraded;
+  int errors;
+
+  //This should be a config. set it to 5 otherwise.
+  static final int poolSize = 5;
+  
+  List<UpgradeExecutor> completedList = new LinkedList<UpgradeExecutor>();
+
+  /* this is set when the datanode misses the regular upgrade.
+   * When this is set, it upgrades the block by reading metadata from
+   * the other replicas.
+   */
+  boolean offlineUpgrade = false;
+  boolean upgradeCompleted = false;
+  
+  boolean isOfflineUpgradeOn() {
+    return offlineUpgrade;
+  }
+  
+  // common upgrade interface:
+  
+  public int getVersion() {
+    return BlockCrcUpgradeUtils.PRE_BLOCK_CRC_LAYOUT_VERSION;
+  }
+
+  /*
+   * Start upgrade if it not already running. It sends status to
+   * namenode even if an upgrade is already in progress.
+   */
+  public synchronized UpgradeCommand startUpgrade() throws IOException {
+
+    if ( offlineUpgrade ) {
+      //run doUpgrade here.
+      doUpgrade();
+    }
+    
+    return null; 
+  }
+
+  
+  @Override
+  public String getDescription() {
+    return "Block CRC Upgrade at Datanode";
+  }
+
+  @Override
+  public short getUpgradeStatus() {
+    return (blocksToUpgrade == blocksUpgraded) ? 100 :
+      (short) Math.floor(blocksUpgraded*100.0/blocksToUpgrade);
+  }
+
+  public UpgradeCommand completeUpgrade() throws IOException {
+    // return latest stats command.
+    assert getUpgradeStatus() == 100;
+    return new BlockCrcUpgradeUtils.
+               DatanodeStatsCommand(getUpgradeStatus(),                                 
+                                    getDatanode().dnRegistration,
+                                    blocksPreviouslyUpgraded + blocksUpgraded,
+                                    blocksToUpgrade-blocksUpgraded,
+                                    errors); 
+  }
+  
+  
+  // see description for super.preUpgradeAction().
+  @Override
+  boolean preUpgradeAction(NamespaceInfo nsInfo) throws IOException {
+    int nsUpgradeVersion = nsInfo.getDistributedUpgradeVersion();
+    if(nsUpgradeVersion >= getVersion()) {
+      return false; // Normal upgrade.
+    }
+    
+    LOG.info("\n  This Datanode has missed a cluster wide Block CRC Upgrade." +
+             "\n  Will perform an 'offline' upgrade of the blocks." +
+             "\n  During this time, Datanode does not heartbeat.");
+    
+    DataNode dataNode = getDatanode();
+    
+    //Make sure namenode removes this node from the registered nodes
+    try {
+      // Should we add another error type? Right now only DISK_ERROR removes it
+      // from node list. 
+      dataNode.namenode.errorReport(dataNode.dnRegistration,
+                                    DatanodeProtocol.NOTIFY, 
+                                    "Performing an offline upgrade. " +
+                                    "Will be back online once the ugprade " +
+                                    "completes. Please see datanode logs.");
+      
+    } catch(IOException ignored) {}
+    
+    offlineUpgrade = true;
+    return true;
+  }
+
+  public BlockCrcUpgradeObjectDatanode() {
+  }
+  
+  class UpgradeExecutor implements Runnable {
+    Block block;
+    Throwable throwable;
+    
+    UpgradeExecutor( Block b ) {
+      block = b;
+    }
+    public void run() {
+      try {
+        DNBlockUpgradeInfo blockInfo = new DNBlockUpgradeInfo();
+        blockInfo.block = block;
+        blockInfo.dataNode = getDatanode();
+        blockInfo.namenode = namenode;
+        blockInfo.offlineUpgrade = offlineUpgrade;
+        BlockCrcUpgradeUtils.upgradeBlock( blockInfo );
+      } catch ( Throwable t ) {
+        throwable = t;
+      }
+      synchronized (completedList) {
+        completedList.add( this );
+        completedList.notify();
+      }
+    }
+  }
+  
+  void doUpgrade() throws IOException {
+    
+    if ( upgradeCompleted ) {
+      assert offlineUpgrade : 
+             ("Multiple calls to doUpgrade is expected only during " +
+              "offline upgrade");
+      return;
+    }
+    
+    FSDataset dataset = getDatanode().data;
+
+    // Set up the retry policy so that each attempt waits for one minute.
+    Configuration conf = new Configuration();
+    // set rpc timeout to one minute.
+    conf.set("ipc.client.timeout", "60000");
+    
+    RetryPolicy timeoutPolicy = 
+       RetryPolicies.retryUpToMaximumCountWithFixedSleep(
+               BlockCrcUpgradeUtils.LONG_TIMEOUT_MINUTES,
+               1, TimeUnit.MILLISECONDS);
+
+    Map<Class<? extends Exception>,RetryPolicy> exceptionToPolicyMap =
+      new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(SocketTimeoutException.class, timeoutPolicy);
+    RetryPolicy methodPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+    Map<String,RetryPolicy> methodNameToPolicyMap = 
+                            new HashMap<String, RetryPolicy>();
+    // do we need to set the policy for connection failures also? 
+    methodNameToPolicyMap.put("processUpgradeCommand", methodPolicy);
+    methodNameToPolicyMap.put("blockCrcUpgradeGetBlockLocations", methodPolicy);
+    
+    LOG.info("Starting Block CRC upgrade.");
+    
+    namenode = (DatanodeProtocol) RetryProxy.create(
+                    DatanodeProtocol.class,
+                    RPC.waitForProxy(DatanodeProtocol.class,
+                                     DatanodeProtocol.versionID,
+                                     getDatanode().getNameNodeAddr(),
+                                     conf),
+                    methodNameToPolicyMap);
+                                  
+    conf = null;
+   
+    // Get a list of all the blocks :
+   
+    LinkedList<UpgradeExecutor> blockList = new LinkedList<UpgradeExecutor>();
+    
+    //Fill blockList with blocks to be upgraded.
+    Block [] blockArr = dataset.getBlockReport();
+    
+    for ( Block b : blockArr ) {
+      File blockFile = dataset.getBlockFile( b );
+      if (!blockFile.exists()) {
+        LOG.error("could not find block file " + blockFile);
+        errors++;
+        continue;
+      }
+      File metaFile = FSDataset.getMetaFile( blockFile );
+      if ( metaFile.exists() ) {
+        blocksPreviouslyUpgraded++;
+      } else {
+        blocksToUpgrade++;
+        blockList.add( new UpgradeExecutor( b ) );
+      }
+    }
+    blockArr = null;
+    
+    ExecutorService pool = Executors.newFixedThreadPool( poolSize );
+    
+    LOG.info("Starting upgrade of " + blocksToUpgrade + " blocks out of " +
+             (blocksToUpgrade + blocksPreviouslyUpgraded));
+    
+    // Do we need to do multiple retries for each upgrade?
+    
+    for (Iterator<UpgradeExecutor> it = blockList.iterator(); it.hasNext();) {
+      pool.submit( it.next() );
+    }
+
+    // Inform the namenode
+    sendStatus();
+    
+    // Now wait for the tasks to complete.
+    int nLeft = blockList.size();
+    
+    long now = System.currentTimeMillis();
+    // Report status to namenode every so many seconds:
+    long statusReportIntervalMilliSec = 60*1000;
+    long lastStatusReportTime = now;
+    long lastUpdateTime = now;
+    long lastWarnTime = now;
+    
+    while ( nLeft > 0 ) {
+      synchronized ( completedList ) {
+        if ( completedList.size() <= 0 ) {
+          try {
+            completedList.wait(1000);
+          } catch (InterruptedException ignored) {}
+        }
+        
+        now = System.currentTimeMillis();
+        
+        if ( completedList.size() > 0 ) {
+          UpgradeExecutor exe = completedList.remove(0);
+          nLeft--;
+          if ( exe.throwable != null ) {
+            errors++;
+            LOG.error("Got an exception during upgrade of " +
+                      exe.block + ": " + 
+                      StringUtils.stringifyException( exe.throwable ));
+          }
+          blocksUpgraded++;
+          lastUpdateTime = now;
+        } else {
+          if ((now - lastUpdateTime) >= 
+                BlockCrcUpgradeUtils.LONG_TIMEOUT_MILLISEC &&
+              (now - lastWarnTime) >= 
+                BlockCrcUpgradeUtils.LONG_TIMEOUT_MILLISEC) {
+            lastWarnTime = now;
+            LOG.warn("No block was updated in last " +
+                      BlockCrcUpgradeUtils.LONG_TIMEOUT_MINUTES +
+                      " minutes! will keep waiting... ");
+          }  
+        } 
+      }
+      
+      if ( (now-lastStatusReportTime) > statusReportIntervalMilliSec ) {
+        sendStatus();
+        lastStatusReportTime = System.currentTimeMillis();
+      }
+    }
+
+    upgradeCompleted = true;
+    
+    LOG.info("Completed BlockCrcUpgrade. total of " + 
+             (blocksPreviouslyUpgraded + blocksToUpgrade) +
+             " blocks : " + blocksPreviouslyUpgraded + " blocks previously " +
+             "upgraded, " + blocksUpgraded + " blocks upgraded this time " +
+             "with " + errors + " errors.");       
+
+    // now inform the name node about the completion.
+
+    // What if there is no upgrade running on Namenode now?
+    while (!sendStatus());
+    
+  }
+  
+  /** Sends current status and stats to namenode and logs it to local log*/ 
+  boolean sendStatus() {
+    BlockCrcUpgradeUtils.DatanodeStatsCommand cmd = null;
+    
+    LOG.info((offlineUpgrade ? "Offline " : "") + "Block CRC Upgrade : " + 
+             getUpgradeStatus() + "% completed.");
+    
+    if (offlineUpgrade) {
+      return true;
+    }
+    
+    synchronized (this) {
+      cmd = new BlockCrcUpgradeUtils.
+      DatanodeStatsCommand(getUpgradeStatus(),                                 
+                           getDatanode().dnRegistration,
+                           blocksPreviouslyUpgraded + blocksUpgraded,
+                           blocksToUpgrade-blocksUpgraded,
+                           errors);
+    }
+
+    UpgradeCommand reply = BlockCrcUpgradeUtils.sendCommand(namenode, cmd, 0);
+    if ( reply == null ) {
+      LOG.warn("Could not send status to Namenode. Namenode might be " +
+               "over loaded or down.");
+    }
+
+    // its ok even if reply == noUpgradeOnNamenode
+    return reply != null;
+  }
+}
+
+/**
+ * Once an upgrade starts at the namenode , this class manages the upgrade 
+ * process.
+ */
+class BlockCrcUpgradeObjectNamenode extends UpgradeObjectNamenode {
+  
+  public static final Log LOG = 
+    LogFactory.getLog("org.apache.hadoop.dfs.BlockCrcUpgradeNamenode");
+  
+  static final long inactivityExtension = 10*1000; // 10 seconds
+  long lastNodeCompletionTime = 0;
+  
+  enum UpgradeStatus {
+    INITIALIZED,
+    STARTED,
+    DATANODES_DONE,
+    COMPLETED,
+  }
+  
+  UpgradeStatus upgradeStatus = UpgradeStatus.INITIALIZED;
+  
+  class DnInfo { 
+    short percentCompleted = 0;
+    long blocksUpgraded = 0;
+    long blocksRemaining = 0;
+    long errors = 0;
+    
+    DnInfo(short pcCompleted) {
+      percentCompleted = status;
+    }
+    DnInfo() {}
+    
+    void setStats(BlockCrcUpgradeUtils.DatanodeStatsCommand cmd) {
+      percentCompleted = cmd.getCurrentStatus();
+      blocksUpgraded = cmd.blocksUpgraded;
+      blocksRemaining = cmd.blocksRemaining;
+      errors = cmd.errors;
+    }
+    
+    boolean isDone() {
+      return percentCompleted >= 100;
+    }
+  }
+  
+  /* We should track only the storageIDs and not DatanodeID, which
+   * includes datanode name and storage id.
+   */
+  HashMap<DatanodeID, DnInfo> dnMap = new HashMap<DatanodeID, DnInfo>();
+  HashMap<DatanodeID, DnInfo> unfinishedDnMap = 
+                                      new HashMap<DatanodeID, DnInfo>();  
+
+  Daemon monitorThread;
+  double avgDatanodeCompletionPct = 0;
+  
+  boolean forceDnCompletion = false;
+  
+  //Upgrade object interface:
+  
+  public int getVersion() {
+    return BlockCrcUpgradeUtils.PRE_BLOCK_CRC_LAYOUT_VERSION;
+  }
+
+  public UpgradeCommand completeUpgrade() throws IOException {
+    return null;
+  }
+ 
+  @Override
+  public String getDescription() {
+    return "Block CRC Upgrade at Namenode"; 
+  }
+
+  @Override
+  public synchronized short getUpgradeStatus() {
+    // Reserve 10% for deleting files.
+    if ( upgradeStatus == UpgradeStatus.COMPLETED ) {
+      return 100;
+    }   
+    if ( upgradeStatus == UpgradeStatus.DATANODES_DONE ) {
+      return 90;
+    }
+    
+    return (short) Math.floor(avgDatanodeCompletionPct * 0.9);
+  }
+
+  public UpgradeCommand startUpgrade() throws IOException {
+    
+    assert monitorThread == null;
+    
+    lastNodeCompletionTime = System.currentTimeMillis();
+    
+    monitorThread = new Daemon(new UpgradeMonitor());
+    monitorThread.start();    
+    
+    return super.startUpgrade();
+  }
+  
+  @Override
+  public synchronized void forceProceed() throws IOException {    
+    if (isUpgradeDone() || 
+        upgradeStatus == UpgradeStatus.DATANODES_DONE) {
+      LOG.info("forceProceed is a no op now since the stage waiting " +
+               "waiting for Datanode to completed is finished. " +
+               "Upgrade should soon complete");
+      return;
+    }
+    
+    if (forceDnCompletion) {
+      LOG.warn("forceProceed is already set for this upgrade. It can take " +
+               "a short while to take affect. Please wait.");
+      return;
+    }
+    
+    LOG.info("got forceProceed request for this upgrade. Datanodes upgrade " +
+             "will be considered done. It can take a few seconds to take " +
+             "effect.");
+    forceDnCompletion = true;
+  }
+
+  @Override
+  UpgradeCommand processUpgradeCommand(UpgradeCommand command) 
+                                           throws IOException {
+    switch (command.getAction()) {
+
+    case BlockCrcUpgradeUtils.DN_CMD_CRC_INFO :
+      return handleCrcInfoCmd(command);
+    
+    case BlockCrcUpgradeUtils.DN_CMD_STATS :
+      return handleStatsCmd(command);
+
+     default:
+       throw new IOException("Unknown Command for BlockCrcUpgrade : " +
+                             command.getAction());
+    }
+  }
+
+  @Override
+  public UpgradeStatusReport getUpgradeStatusReport(boolean details) 
+                                                    throws IOException {
+
+    /* If 'details' is true should we update block level status?
+     * It could take multiple minutes
+     * updateBlckLevelStats()?
+     */
+    
+    String replyString = "";
+    
+    short status = 0;
+    
+    synchronized (this) {
+     
+      status = getUpgradeStatus();
+     
+      replyString = String.format(
+      ((monitorThread == null) ? "\tUpgrade has not been started yet.\n" : "")+
+      ((forceDnCompletion) ? "\tForce Proceed is ON\n" : "") +
+      "\tLast Block Level Stats updated at : %tc\n" +
+      "\tLast Block Level Stats : %s\n" +
+      "\tBrief Datanode Status  : %s\n" +
+      "%s",
+      latestBlockLevelStats.updatedAt,
+      latestBlockLevelStats.statusString("\n\t                         "), 
+      printStatus("\n\t                         "), 
+      ((status < 100 && upgradeStatus == UpgradeStatus.DATANODES_DONE) ?
+      "\tNOTE: Upgrade at the Datanodes has finished. Deleteing \".crc\" " +
+      "files\n\tcan take longer than status implies.\n" : "")
+      );
+      
+      if (details) {
+        // list all the known data nodes
+        StringBuilder str = null;
+        Iterator<DatanodeID> keys = dnMap.keySet().iterator();
+        Iterator<DnInfo> values = dnMap.values().iterator();
+        
+        for(; keys.hasNext() && values.hasNext() ;) {
+          DatanodeID dn = keys.next();
+          DnInfo info = values.next();
+          String dnStr = "\t\t" + dn.getName() + "\t : " + 
+                         info.percentCompleted + " % \t" +
+                         info.blocksUpgraded + " u \t" +
+                         info.blocksRemaining + " r \t" +
+                         info.errors + " e\n";
+          if ( str == null ) {
+            str = new StringBuilder(dnStr.length()*
+                                    (dnMap.size() + (dnMap.size()+7)/8));
+          }
+          str.append(dnStr);
+        }
+        
+        replyString += "\n\tDatanode Stats (total: " + dnMap.size() + "): " +
+                       "pct Completion(%) blocks upgraded (u) " +
+                       "blocks remaining (r) errors (e)\n\n" +
+                       (( str == null ) ?
+                        "\t\tThere are no known Datanodes\n" : str);
+      }      
+    }
+    
+    return new BlockCrcUpgradeUtils.BlockCrcUpgradeStatusReport(
+                   BlockCrcUpgradeUtils.PRE_BLOCK_CRC_LAYOUT_VERSION,
+                   status, replyString);
+  }
+
+  private UpgradeCommand handleCrcInfoCmd(UpgradeCommand cmd) {
+    BlockCrcUpgradeUtils.CrcInfoCommand crcCmd =
+                   (BlockCrcUpgradeUtils.CrcInfoCommand)cmd;
+    
+    BlockCrcInfo crcInfo = getFSNamesystem().blockCrcInfo(crcCmd.block,
+                                                          false);
+    return new BlockCrcUpgradeUtils.CrcInfoCommandReply(crcInfo);
+  }
+  
+  private synchronized UpgradeCommand handleStatsCmd(UpgradeCommand cmd) {
+    
+    BlockCrcUpgradeUtils.DatanodeStatsCommand stats =
+      (BlockCrcUpgradeUtils.DatanodeStatsCommand)cmd;
+    
+    DatanodeID dn = stats.datanodeId;
+    DnInfo dnInfo = dnMap.get(dn);
+    boolean alreadyCompleted = (dnInfo != null && dnInfo.isDone());
+    
+    if ( dnInfo == null ) {
+      dnInfo = new DnInfo();
+      dnMap.put(dn, dnInfo);
+      LOG.info("Upgrade started/resumed at datanode " + dn.getName());  
+    }
+    
+    dnInfo.setStats(stats);
+
+    if ( !dnInfo.isDone() ) {
+      unfinishedDnMap.put(dn, dnInfo);
+    }
+    
+    if ( dnInfo.isDone() && !alreadyCompleted ) {
+      LOG.info("upgrade completed on datanode " + dn.getName());      
+      unfinishedDnMap.remove(dn);
+      if (unfinishedDnMap.size() == 0) {
+        lastNodeCompletionTime = System.currentTimeMillis();
+      }
+    }   
+    
+    //Should we send any more info?
+    return new UpgradeCommand();
+  }
+  
+  public BlockCrcUpgradeObjectNamenode() {
+  }
+  
+  // For now we will wait for all the nodes to complete upgrade.
+  synchronized boolean isUpgradeDone() {
+    return upgradeStatus == UpgradeStatus.COMPLETED;    
+  }
+  
+  synchronized String printStatus(String spacing) {
+    //NOTE: iterates on all the datanodes.
+    
+    // Calculate % completion on all the data nodes.
+    long errors = 0;
+    long totalCompletion = 0;
+    for( Iterator<DnInfo> it = dnMap.values().iterator(); it.hasNext(); ) {
+      DnInfo dnInfo = it.next();
+      totalCompletion += dnInfo.percentCompleted;            
+      errors += dnInfo.errors;
+    }
+    
+    avgDatanodeCompletionPct = totalCompletion/(dnMap.size() + 1e-20);
+    
+    String msg = "Avg completion on Datanodes: " +              
+                 String.format("%.2f%%", avgDatanodeCompletionPct) +
+                 " with " + errors + " errors. " +
+                 ((unfinishedDnMap.size() > 0) ? spacing + 
+                   unfinishedDnMap.size() + " out of " + dnMap.size() +
+                   " nodes are not done." : "");
+                 
+    LOG.info("Upgrade " + (isUpgradeDone() ? 
+             "completed. " : "still running. ") + spacing + msg);
+    return msg;
+  }
+  
+  private synchronized void setStatus(UpgradeStatus status) {
+    upgradeStatus = status;
+  }
+
+  /* Checks if upgrade completed based on datanode's status and/or 
+   * if all the blocks are upgraded.
+   */
+  private synchronized UpgradeStatus checkOverallCompletion() {
+    
+    if (upgradeStatus == UpgradeStatus.COMPLETED ||
+        upgradeStatus == UpgradeStatus.DATANODES_DONE) {
+      return upgradeStatus;
+    }
+    
+    if (upgradeStatus != UpgradeStatus.DATANODES_DONE) {
+      boolean datanodesDone =
+        (dnMap.size() > 0 && unfinishedDnMap.size() == 0 &&
+         ( System.currentTimeMillis() - lastNodeCompletionTime ) > 
+        inactivityExtension) || forceDnCompletion ;
+                 
+      if ( datanodesDone ) {
+        LOG.info(((forceDnCompletion) ? "Forcing " : "Marking ") +
+                 "Datanode Upgrade done");
+        upgradeStatus = UpgradeStatus.DATANODES_DONE;
+      }
+    }
+    
+    if (upgradeStatus != UpgradeStatus.DATANODES_DONE &&
+        latestBlockLevelStats.updatedAt > 0) {
+      // check if last block report marked all
+      if (latestBlockLevelStats.minimallyReplicatedBlocks == 0 &&
+          latestBlockLevelStats.underReplicatedBlocks == 0) {
+        
+        LOG.info("Marking datanode upgrade complete since all the blocks are " +
+                 "upgraded (even though some datanodes may not have " +
+                 "reported completion. Block level stats :\n\t" +
+                 latestBlockLevelStats.statusString("\n\t"));
+        upgradeStatus = UpgradeStatus.DATANODES_DONE;
+      }
+    }
+    
+    return upgradeStatus;
+  } 
+    
+  /**
+   * This class monitors the upgrade progress and periodically prints 
+   * status message to log.
+   */
+  class UpgradeMonitor implements Runnable {
+    
+    static final long statusReportIntervalMillis = 1*60*1000;
+    static final long blockReportIntervalMillis = 5*60*1000;
+    static final int sleepTimeSec = 1;
+    
+    public void run() {
+      long lastReportTime = System.currentTimeMillis();
+      long lastBlockReportTime = lastReportTime;
+      
+      while ( !isUpgradeDone() ) {
+        UpgradeStatus status = checkOverallCompletion();
+        
+        if ( status == UpgradeStatus.DATANODES_DONE ) {
+          deleteCrcFiles();
+          setStatus(UpgradeStatus.COMPLETED);
+        }
+        
+        long now = System.currentTimeMillis();
+        
+        
+        if (now-lastBlockReportTime >= blockReportIntervalMillis) {
+          updateBlockLevelStats();
+          // Check if all the blocks have been upgraded.
+          lastBlockReportTime = now;
+        }
+        
+        if ((now - lastReportTime) >= statusReportIntervalMillis || 
+            isUpgradeDone()) {
+          printStatus("\n\t");
+          lastReportTime = now;
+        }
+        
+        BlockCrcUpgradeUtils.sleep(sleepTimeSec, null);
+      }
+      
+      LOG.info("Leaving the monitor thread");
+    }
+  }
+  
+  private BlockLevelStats latestBlockLevelStats = new BlockLevelStats();
+  // internal class to hold the stats.
+  private static class BlockLevelStats {
+    long fullyReplicatedBlocks = 0;
+    long minimallyReplicatedBlocks = 0;
+    long underReplicatedBlocks = 0; // includes unReplicatedBlocks
+    long unReplicatedBlocks = 0; // zero replicas upgraded
+    long errors;
+    long updatedAt;
+    
+    String statusString(String spacing) {
+      long totalBlocks = fullyReplicatedBlocks + 
+                         minimallyReplicatedBlocks +
+                         underReplicatedBlocks;
+      double multiplier = 100/(totalBlocks + 1e-20);
+      
+      if (spacing.equals("")) {
+        spacing = ", ";
+      }
+      
+      return String.format(
+                     "Total Blocks : %d" +
+                     "%sFully Upgragraded : %.2f%%" +
+                     "%sMinimally Upgraded : %.2f%%" +
+                     "%sUnder Upgraded : %.2f%% (includes Un-upgraded blocks)" +
+                     "%sUn-upgraded : %.2f%%" + 
+                     "%sErrors : %d", totalBlocks, 
+                     spacing, (fullyReplicatedBlocks * multiplier),
+                     spacing, (minimallyReplicatedBlocks * multiplier),
+                     spacing, (underReplicatedBlocks * multiplier),
+                     spacing, (unReplicatedBlocks * multiplier),
+                     spacing, errors);
+    }
+  }
+  
+  void updateBlockLevelStats(String path, BlockLevelStats stats) {
+    DFSFileInfo[] fileArr = getFSNamesystem().getListing(path);
+    
+    for (DFSFileInfo file:fileArr) {
+      if (file.isDir()) {
+        updateBlockLevelStats(file.getPath(), stats);
+      } else {
+        // Get the all the blocks.
+        LocatedBlocks blockLoc = null;
+        try {
+          blockLoc = getFSNamesystem().getBlockLocations(null, file.getPath(), 0, 
+                                                  file.getLen());
+          int numBlocks = blockLoc.locatedBlockCount();
+          for (int i=0; i<numBlocks; i++) {
+            LocatedBlock loc = blockLoc.get(i);
+            DatanodeInfo[] dnArr = loc.getLocations();
+            int numUpgraded = 0;
+            synchronized (this) {
+              for (DatanodeInfo dn:dnArr) {
+                DnInfo dnInfo = dnMap.get(dn);
+                if (dnInfo != null && dnInfo.isDone()) {
+                  numUpgraded++;
+                }
+              }
+            }
+            
+            if (numUpgraded >= file.getReplication()) {
+              stats.fullyReplicatedBlocks++;
+            } else if (numUpgraded >= getFSNamesystem().getMinReplication()) {
+              stats.minimallyReplicatedBlocks++;
+            } else {
+              stats.underReplicatedBlocks++;
+            }
+            if (numUpgraded == 0) {
+              stats.unReplicatedBlocks++;
+            }
+          }
+        } catch (IOException e) {
+          LOG.error("BlockCrcUpgrade: could not get block locations for " +
+                    file.getPath() + " : " + StringUtils.stringifyException(e));
+          stats.errors++;
+        }
+      }
+    }
+  }
+  
+  void updateBlockLevelStats() {
+    /* This iterates over all the blocks and updates various 
+     * counts.
+     * Since iterating over all the blocks at once would be quite 
+     * large operation under lock, we iterate over all the files
+     * and update the counts for blocks that belong to a file.
+     */
+      
+    LOG.info("Starting update of block level stats. " +
+             "This could take a few minutes");
+    BlockLevelStats stats = new BlockLevelStats();
+    updateBlockLevelStats("/", stats);
+    stats.updatedAt = System.currentTimeMillis();
+    
+    LOG.info("Block level stats:\n\t" + stats.statusString("\n\t"));
+    synchronized (this) {
+      latestBlockLevelStats = stats;
+    }
+  }
+  
+  private int deleteCrcFiles(String path) {
+    // Recursively deletes files
+    DFSFileInfo[] fileArr = getFSNamesystem().getListing(path);
+    
+    int numFilesDeleted = 0;
+    
+    HashSet<String> fileSet = new HashSet<String>();
+    
+    // build a small hashMap
+    for ( DFSFileInfo file:fileArr ) {
+      String name = file.getName();
+      if (!file.isDir() && (!name.startsWith(".") || 
+                            !name.endsWith(".crc"))) {
+        fileSet.add(name);
+      }
+    }
+    
+    for ( DFSFileInfo file:fileArr ) {
+      if ( !file.isDir() ) {
+        String name = file.getName();
+        int extraLen = ".".length() + ".crc".length();
+        if (name.startsWith(".") && name.endsWith(".crc") && 
+            name.length() > extraLen) {
+          String dataFile = name.substring(1, name.length()-extraLen+1);
+          
+          /* Deleting many files at once might be too much load on namenode,
+           * especially on large clusters. We could throttle based on
+           * some namesystem state. We can set high-low watermarks for 
+           * pending deletes, etc.
+           */
+          if (fileSet.contains(dataFile)) {
+            String filepath = path + (path.endsWith("/") ? "" : "/") + name;
+            
+            try {
+              LOG.debug("Deleting " + filepath);
+              if (getFSNamesystem().deleteInSafeMode(filepath)) {
+                numFilesDeleted++;
+              }
+            } catch (IOException e) {
+              LOG.error("Exception while deleting " + filepath + 
+                        " : " + StringUtils.stringifyException(e));
+            }
+          }
+        }
+      }
+    }
+    
+    // Reduce memory before recursion
+    fileSet = null;
+    
+    for ( DFSFileInfo file:fileArr ) {
+      if ( file.isDir() ) {
+        numFilesDeleted += deleteCrcFiles(file.getPath());
+      }
+    }
+    
+    return numFilesDeleted;
+  }
+  
+  int deleteCrcFiles() {
+    /* Iterate over all the files and delete any file with name .fname.crc
+     * if fname exists in the directory.
+     */
+    while (true) {
+      LOG.info("Deleting \".crc\" files. This can take few minutes ... ");
+      int numFilesDeleted = deleteCrcFiles("/");
+      LOG.info("Deleted " + numFilesDeleted + " files");
+      break;
+      // Should we iterate again? No need for now!
+    }
+    return 0;
+  } 
+}

+ 131 - 0
src/java/org/apache/hadoop/dfs/ChecksumDistributedFileSystem.java

@@ -0,0 +1,131 @@
+/**
+ * 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.dfs;
+
+import java.io.*;
+import java.net.*;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.dfs.FSConstants.UpgradeAction;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * An implementation of ChecksumFileSystem over DistributedFileSystem. 
+ * Note that as of now (May 07), DistributedFileSystem natively checksums 
+ * all of its data. Using this class is not be necessary in most cases.
+ * Currently provided mainly for backward compatibility and testing.
+ */
+public class ChecksumDistributedFileSystem extends ChecksumFileSystem {
+  
+  public ChecksumDistributedFileSystem() {
+    super( new DistributedFileSystem() );
+  }
+
+  /** @deprecated */
+  public ChecksumDistributedFileSystem(InetSocketAddress namenode,
+                                       Configuration conf) throws IOException {
+    super( new DistributedFileSystem(namenode, conf) );
+  }
+  
+  /** Any extra interface that DistributeFileSystem provides can be
+   * accessed with this.*/
+  DistributedFileSystem getDFS() {
+    return (DistributedFileSystem)fs;
+  }
+
+  @Override
+  public long getContentLength(Path f) throws IOException {
+    return fs.getContentLength(f);
+  }
+
+  /** Return the total raw capacity of the filesystem, disregarding
+   * replication .*/
+  public long getRawCapacity() throws IOException{
+    return getDFS().getRawCapacity();
+  }
+
+  /** Return the total raw used space in the filesystem, disregarding
+   * replication .*/
+  public long getRawUsed() throws IOException{
+    return getDFS().getRawUsed();
+  }
+
+  /** Return statistics for each datanode. */
+  public DatanodeInfo[] getDataNodeStats() throws IOException {
+    return getDFS().getDataNodeStats();
+  }
+    
+  /**
+   * Enter, leave or get safe mode.
+   *  
+   * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
+   */
+  public boolean setSafeMode(FSConstants.SafeModeAction action) 
+    throws IOException {
+    return getDFS().setSafeMode(action);
+  }
+
+  /*
+   * Refreshes the list of hosts and excluded hosts from the configured 
+   * files.  
+   */
+  public void refreshNodes() throws IOException {
+    getDFS().refreshNodes();
+  }
+
+  /**
+   * Finalize previously upgraded files system state.
+   */
+  public void finalizeUpgrade() throws IOException {
+    getDFS().finalizeUpgrade();
+  }
+
+  public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
+                                                        ) throws IOException {
+    return getDFS().distributedUpgradeProgress(action);
+  }
+
+  /*
+   * Dumps dfs data structures into specified file.
+   */
+  public void metaSave(String pathname) throws IOException {
+    getDFS().metaSave(pathname);
+  }
+
+  /**
+   * We need to find the blocks that didn't match.  Likely only one 
+   * is corrupt but we will report both to the namenode.  In the future,
+   * we can consider figuring out exactly which block is corrupt.
+   */
+  public boolean reportChecksumFailure(Path f, 
+                                       FSDataInputStream in, long inPos, 
+                                       FSDataInputStream sums, long sumsPos) {
+    return getDFS().reportChecksumFailure(f, in, inPos, sums, sumsPos);
+  }
+  
+  
+  /**
+   * Returns the stat information about the file.
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    return getDFS().getFileStatus(f);
+  }
+
+}

Rozdielové dáta súboru neboli zobrazené, pretože súbor je príliš veľký
+ 456 - 258
src/java/org/apache/hadoop/dfs/DFSClient.java


+ 234 - 0
src/java/org/apache/hadoop/dfs/DataChecksum.java

@@ -0,0 +1,234 @@
+/**
+ * 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.dfs;
+
+import java.util.zip.Checksum;
+import java.util.zip.CRC32;
+
+import java.io.*;
+
+/**
+ * This class provides inteface and utilities for processing checksums for
+ * DFS data transfers.
+ */
+
+public class DataChecksum implements Checksum {
+  
+  // Misc constants
+  public static final int HEADER_LEN = 5; /// 1 byte type and 4 byte len
+  
+  // checksum types
+  public static final int CHECKSUM_NULL    = 0;
+  public static final int CHECKSUM_CRC32   = 1;
+  
+  private static final int CHECKSUM_NULL_SIZE  = 0;
+  private static final int CHECKSUM_CRC32_SIZE = 4;
+  
+  
+  public static DataChecksum newDataChecksum( int type, int bytesPerChecksum ) {
+    if ( bytesPerChecksum <= 0 ) {
+      return null;
+    }
+    
+    switch ( type ) {
+    case CHECKSUM_NULL :
+      return new DataChecksum( CHECKSUM_NULL, new ChecksumNull(), 
+                               CHECKSUM_NULL_SIZE, bytesPerChecksum );
+    case CHECKSUM_CRC32 :
+      return new DataChecksum( CHECKSUM_CRC32, new CRC32(), 
+                               CHECKSUM_CRC32_SIZE, bytesPerChecksum );
+    default:
+      return null;  
+    }
+  }
+  
+  /**
+   * Creates a DataChecksum from HEADER_LEN bytes from arr[offset].
+   * @return DataChecksum of the type in the array or null in case of an error.
+   */
+  public static DataChecksum newDataChecksum( byte bytes[], int offset ) {
+    if ( offset < 0 || bytes.length < offset + HEADER_LEN ) {
+      return null;
+    }
+    
+    // like readInt():
+    int bytesPerChecksum = ( (bytes[offset+1] & 0xff) << 24 ) | 
+                           ( (bytes[offset+2] & 0xff) << 16 ) |
+                           ( (bytes[offset+3] & 0xff) << 8 )  |
+                           ( (bytes[offset+4] & 0xff) );
+    return newDataChecksum( bytes[0], bytesPerChecksum );
+  }
+  
+  /**
+   * This constructucts a DataChecksum by reading HEADER_LEN bytes from
+   * input stream <i>in</i>
+   */
+  public static DataChecksum newDataChecksum( DataInputStream in )
+                                 throws IOException {
+    int type = in.readByte();
+    int bpc = in.readInt();
+    DataChecksum summer = newDataChecksum( type, bpc );
+    if ( summer == null ) {
+      throw new IOException( "Could not create DataChecksum of type " +
+                             type + " with bytesPerChecksum " + bpc );
+    }
+    return summer;
+  }
+  
+  /**
+   * Writes the checksum header to the output stream <i>out</i>.
+   */
+  public void writeHeader( DataOutputStream out ) 
+                           throws IOException { 
+    out.writeByte( type );
+    out.writeInt( bytesPerChecksum );
+  }
+  
+  /**
+   * Writes the current checksum to the stream.
+   * If <i>reset</i> is true, then resets the checksum.
+   * @return number of bytes written. Will be equal to getChecksumSize();
+   */
+   public int writeValue( DataOutputStream out, boolean reset )
+                          throws IOException {
+     if ( size <= 0 ) {
+       return 0;
+     }
+
+     if ( type == CHECKSUM_CRC32 ) {
+       out.writeInt( (int) summer.getValue() );
+     } else {
+       throw new IOException( "Unknown Checksum " + type );
+     }
+     
+     if ( reset ) {
+       reset();
+     }
+     
+     return size;
+   }
+   
+   /**
+    * Writes the current checksum to a buffer.
+    * If <i>reset</i> is true, then resets the checksum.
+    * @return number of bytes written. Will be equal to getChecksumSize();
+    */
+    public int writeValue( byte[] buf, int offset, boolean reset )
+                           throws IOException {
+      if ( size <= 0 ) {
+        return 0;
+      }
+
+      if ( type == CHECKSUM_CRC32 ) {
+        int checksum = (int) summer.getValue();
+        buf[offset+0] = (byte) ((checksum >>> 24) & 0xff);
+        buf[offset+1] = (byte) ((checksum >>> 16) & 0xff);
+        buf[offset+2] = (byte) ((checksum >>> 8) & 0xff);
+        buf[offset+3] = (byte) (checksum & 0xff);
+      } else {
+        throw new IOException( "Unknown Checksum " + type );
+      }
+      
+      if ( reset ) {
+        reset();
+      }
+      
+      return size;
+    }
+   
+   /**
+    * Compares the checksum located at buf[offset] with the current checksum.
+    * @return true if the checksum matches and false otherwise.
+    */
+   public boolean compare( byte buf[], int offset ) {
+     if ( size > 0 && type == CHECKSUM_CRC32 ) {
+       int checksum = ( (buf[offset+0] & 0xff) << 24 ) | 
+                      ( (buf[offset+1] & 0xff) << 16 ) |
+                      ( (buf[offset+2] & 0xff) << 8 )  |
+                      ( (buf[offset+3] & 0xff) );
+       return checksum == (int) summer.getValue();
+     }
+     return size == 0;
+   }
+   
+  private final int type;
+  private final int size;
+  private final Checksum summer;
+  private final int bytesPerChecksum;
+  private int inSum = 0;
+  
+  private DataChecksum( int checksumType, Checksum checksum,
+                        int sumSize, int chunkSize ) {
+    type = checksumType;
+    summer = checksum;
+    size = sumSize;
+    bytesPerChecksum = chunkSize;
+  }
+  
+  // Accessors
+  public int getChecksumType() {
+    return type;
+  }
+  public int getChecksumSize() {
+    return size;
+  }
+  public int getBytesPerChecksum() {
+    return bytesPerChecksum;
+  }
+  public int getNumBytesInSum() {
+    return inSum;
+  }
+  //Checksum Interface. Just a wrapper around member summer.
+  public long getValue() {
+    return summer.getValue();
+  }
+  public void reset() {
+    summer.reset();
+    inSum = 0;
+  }
+  public void update( byte[] b, int off, int len ) {
+    if ( len > 0 ) {
+      summer.update( b, off, len );
+      inSum += len;
+    }
+    // Can be removed.
+    assert inSum <= bytesPerChecksum : "DataChecksum.update() : inSum " + 
+                inSum + " > " + " bytesPerChecksum " + bytesPerChecksum ; 
+  }
+  public void update( int b ) {
+    summer.update( b );
+    inSum += 1;
+  }
+  
+  /**
+   * This just provides a dummy implimentation for Checksum class
+   * This is used when there is no checksum available or required for 
+   * data
+   */
+  static class ChecksumNull implements Checksum {
+    
+    public ChecksumNull() {}
+    
+    //Dummy interface
+    public long getValue() { return 0; }
+    public void reset() {}
+    public void update(byte[] b, int off, int len) {}
+    public void update(int b) {}
+  };
+}

+ 475 - 354
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -30,6 +30,9 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.mapred.StatusHttpServer;
 import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.dfs.BlockCommand;
+import org.apache.hadoop.dfs.DatanodeProtocol;
+import org.apache.hadoop.fs.FileUtil;
 
 import java.io.*;
 import java.net.*;
@@ -73,12 +76,13 @@ import org.apache.hadoop.metrics.jvm.JvmMetrics;
  **********************************************************/
 public class DataNode implements FSConstants, Runnable {
   public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.DataNode");
-  //
-  // REMIND - mjc - I might bring "maxgigs" back so user can place 
-  // artificial  limit on space
-  //private static final long GIGABYTE = 1024 * 1024 * 1024;
-  //private static long numGigs = Configuration.get().getLong("dfs.datanode.maxgigs", 100);
-  //
+
+  /**
+   * A buffer size small enough that read/writes while reading headers 
+   * don't result in multiple io calls but reading larger amount of data 
+   * like one checksum size does not result in extra copy. 
+   */
+  public static final int SMALL_HDR_BUFFER_SIZE = 64;
 
   /**
    * Util method to build socket addr from either:
@@ -112,7 +116,7 @@ public class DataNode implements FSConstants, Runnable {
   DatanodeRegistration dnRegistration = null;
   private String networkLoc;
   volatile boolean shouldRun = true;
-  Vector<Block> receivedBlockList = new Vector<Block>();
+  LinkedList<Block> receivedBlockList = new LinkedList<Block>();
   int xmitsInProgress = 0;
   Daemon dataXceiveServer = null;
   long blockReportInterval;
@@ -126,6 +130,7 @@ public class DataNode implements FSConstants, Runnable {
   private static DataNode datanodeObject = null;
   private static Thread dataNodeThread = null;
   String machineName;
+  int defaultBytesPerChecksum = 512;
 
   private static class DataNodeMetrics implements Updater {
     private final MetricsRecord metricsRecord;
@@ -222,6 +227,10 @@ public class DataNode implements FSConstants, Runnable {
                                      conf.get("dfs.datanode.dns.nameserver","default"));
     InetSocketAddress nameNodeAddr = createSocketAddr(
                                                       conf.get("fs.default.name", "local"));
+    
+    this.defaultBytesPerChecksum = 
+       Math.max(conf.getInt("io.bytes.per.checksum", 512), 1); 
+    
     int tmpPort = conf.getInt("dfs.datanode.port", 50010);
     storage = new DataStorage();
     // construct registration
@@ -709,25 +718,30 @@ public class DataNode implements FSConstants, Runnable {
      */
     public void run() {
       try {
-        DataInputStream in = new DataInputStream(new BufferedInputStream(s.getInputStream()));
-        try {
-          byte op = (byte) in.read();
-          if (op == OP_WRITE_BLOCK) {
-            writeBlock(in);
-          } else if (op == OP_READ_BLOCK || op == OP_READSKIP_BLOCK ||
-                     op == OP_READ_RANGE_BLOCK) {
-            readBlock(in, op);
-          } else {
-            while (op >= 0) {
-              System.out.println("Faulty op: " + op);
-              op = (byte) in.read();
-            }
-            throw new IOException("Unknown opcode for incoming data stream");
-          }
-        } finally {
-          in.close();
+        DataInputStream in = new DataInputStream(
+           new BufferedInputStream(s.getInputStream(), SMALL_HDR_BUFFER_SIZE));
+        short version = in.readShort();
+        if ( version != DATA_TRANFER_VERSION ) {
+          throw new IOException( "Version Mismatch" );
+        }
+
+        byte op = in.readByte();
+
+        switch ( op ) {
+        case OP_READ_BLOCK:
+          readBlock( in );
+          break;
+        case OP_WRITE_BLOCK:
+          writeBlock( in );
+          break;
+        case OP_READ_METADATA:
+          readMetadata( in );
+          
+        default:
+          System.out.println("Faulty op: " + op);
+          throw new IOException("Unknown opcode " + op + "in data stream");
         }
-      } catch (Throwable t) {
+       } catch (Throwable t) {
         LOG.error("DataXCeiver", t);
       } finally {
         try {
@@ -742,104 +756,35 @@ public class DataNode implements FSConstants, Runnable {
     /**
      * Read a block from the disk
      * @param in The stream to read from
-     * @param op OP_READ_BLOCK or OP_READ_SKIPBLOCK
      * @throws IOException
      */
-    private void readBlock(DataInputStream in, byte op) throws IOException {
+    private void readBlock(DataInputStream in) throws IOException {
       //
       // Read in the header
       //
-      Block b = new Block();
-      b.readFields(in);
-
-      long toSkip = 0;
-      long endOffset = -1;
-      if (op == OP_READSKIP_BLOCK) {
-        toSkip = in.readLong();
-      } else if (op == OP_READ_RANGE_BLOCK) {
-        toSkip = in.readLong();
-        endOffset = in.readLong();
-      }
+      long blockId = in.readLong();          
+      Block block = new Block( blockId, 0 );
 
-      //
-      // Open reply stream
-      //
-      DataOutputStream out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+      long startOffset = in.readLong();
+      long length = in.readLong();
+      
       try {
-        //
-        // Write filelen of -1 if error
-        //
-        if (!data.isValidBlock(b)) {
-          out.writeLong(-1);
-        } else {
-          //
-          // Get blockdata from disk
-          //
-          long len = data.getLength(b);
-          if (endOffset < 0) { endOffset = len; }
-          DataInputStream in2 = new DataInputStream(data.getBlockData(b));
-          out.writeLong(len);
-
-          long amtSkipped = 0;
-          if ((op == OP_READSKIP_BLOCK) || (op == OP_READ_RANGE_BLOCK)) {
-            if (toSkip > len) {
-              toSkip = len;
-            }
-            try {
-              amtSkipped = in2.skip(toSkip);
-            } catch (IOException iex) {
-              shutdown();
-              throw iex;
-            }
-            out.writeLong(amtSkipped);
-          }
-          if (op == OP_READ_RANGE_BLOCK) {
-            if (endOffset > len) {
-              endOffset = len;
-            }
-            out.writeLong(endOffset);
-          }
-
-          byte buf[] = new byte[BUFFER_SIZE];
-          try {
-            int toRead = (int) (endOffset - amtSkipped + 1);
-            int bytesRead = 0;
-            try {
-              bytesRead = in2.read(buf, 0, Math.min(BUFFER_SIZE, toRead));
-              myMetrics.readBytes(bytesRead);
-            } catch (IOException iex) {
-              shutdown();
-              throw iex;
-            }
-            while (toRead > 0 && bytesRead >= 0) {
-              out.write(buf, 0, bytesRead);
-              toRead -= bytesRead;
-              if (toRead > 0) {
-                try {
-                  bytesRead = in2.read(buf, 0, Math.min(BUFFER_SIZE, toRead));
-                  myMetrics.readBytes(bytesRead);
-                } catch (IOException iex) {
-                  shutdown();
-                  throw iex;
-                }
-              }
-            }
-          } catch (SocketException se) {
-            // This might be because the reader
-            // closed the stream early
-          } finally {
-            try {
-              in2.close();
-            } catch (IOException iex) {
-              shutdown();
-              throw iex;
-            }
-          }
-        }
+        //XXX Buffered output stream?
+        long read = sendBlock(s, block, startOffset, length, null );
+        myMetrics.readBytes((int)read);
         myMetrics.readBlocks(1);
-        LOG.info("Served block " + b + " to " + s.getInetAddress());
-      } finally {
-        out.close();
+        LOG.info("Served block " + block + " to " + s.getInetAddress());
+      } catch ( SocketException ignored ) {
+        // Its ok for remote side to close the connection anytime.
+        myMetrics.readBlocks(1);
+      } catch ( IOException ioe ) {
+        /* What exactly should we do here?
+         * Earlier version shutdown() datanode if there is disk error.
+         */
+        LOG.warn( "Got exception while serving " + block + " to " +
+                  s.getInetAddress() + ": " + 
+                  StringUtils.stringifyException(ioe) );
+        throw ioe;
       }
     }
 
@@ -852,14 +797,23 @@ public class DataNode implements FSConstants, Runnable {
       //
       // Read in the header
       //
-      DataOutputStream reply = 
-        new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
+      DataOutputStream reply = new DataOutputStream(s.getOutputStream());
+      DataOutputStream out = null;
+      DataOutputStream checksumOut = null;
+      Socket mirrorSock = null;
+      DataOutputStream mirrorOut = null;
+      DataInputStream mirrorIn = null;
+      
       try {
-        boolean shouldReportBlock = in.readBoolean();
-        Block b = new Block();
-        b.readFields(in);
+        /* We need an estimate for block size to check if the 
+         * disk partition has enough space. For now we just increment
+         * FSDataset.reserved by configured dfs.block.size
+         * Other alternative is to include the block size in the header
+         * sent by DFSClient.
+         */
+        Block block = new Block( in.readLong(), 0 );
         int numTargets = in.readInt();
-        if (numTargets <= 0) {
+        if ( numTargets < 0 ) {
           throw new IOException("Mislabelled incoming datastream.");
         }
         DatanodeInfo targets[] = new DatanodeInfo[numTargets];
@@ -868,225 +822,419 @@ public class DataNode implements FSConstants, Runnable {
           tmp.readFields(in);
           targets[i] = tmp;
         }
-        byte encodingType = (byte) in.read();
-        long len = in.readLong();
-            
-        //
-        // Make sure curTarget is equal to this machine
-        //
-        DatanodeInfo curTarget = targets[0];
-            
-        //
-        // Track all the places we've successfully written the block
-        //
-        Vector<DatanodeInfo> mirrors = new Vector<DatanodeInfo>();
             
+        DataChecksum checksum = DataChecksum.newDataChecksum( in );
+
         //
         // Open local disk out
         //
-        OutputStream o;
-        try {
-          o = data.writeToBlock(b);
-        } catch( IOException e ) {
-          checkDiskError( e );
-          throw e;
-        }
-        DataOutputStream out = new DataOutputStream(new BufferedOutputStream(o));
+        FSDataset.BlockWriteStreams streams = data.writeToBlock( block );
+        out = new DataOutputStream(streams.dataOut);
+        checksumOut = new DataOutputStream(streams.checksumOut);
+        
         InetSocketAddress mirrorTarget = null;
         String mirrorNode = null;
-        try {
-          //
-          // Open network conn to backup machine, if 
-          // appropriate
-          //
-          DataInputStream in2 = null;
-          DataOutputStream out2 = null;
-          if (targets.length > 1) {
-            // Connect to backup machine
-            mirrorNode = targets[1].getName();
-            mirrorTarget = createSocketAddr(mirrorNode);
-            try {
-              Socket s2 = new Socket();
-              s2.connect(mirrorTarget, READ_TIMEOUT);
-              s2.setSoTimeout(READ_TIMEOUT);
-              out2 = new DataOutputStream(new BufferedOutputStream(s2.getOutputStream()));
-              in2 = new DataInputStream(new BufferedInputStream(s2.getInputStream()));
-                  
-              // Write connection header
-              out2.write(OP_WRITE_BLOCK);
-              out2.writeBoolean(shouldReportBlock);
-              b.write(out2);
-              out2.writeInt(targets.length - 1);
-              for (int i = 1; i < targets.length; i++) {
-                targets[i].write(out2);
-              }
-              out2.write(encodingType);
-              out2.writeLong(len);
-              myMetrics.replicatedBlocks(1);
-            } catch (IOException ie) {
-              if (out2 != null) {
-                LOG.info("Exception connecting to mirror " + mirrorNode 
-                         + "\n" + StringUtils.stringifyException(ie));
-                try {
-                  out2.close();
-                  in2.close();
-                } catch (IOException out2close) {
-                } finally {
-                  out2 = null;
-                  in2 = null;
-                }
-              }
+        //
+        // Open network conn to backup machine, if 
+        // appropriate
+        //
+        if (targets.length > 0) {
+          // Connect to backup machine
+          mirrorNode = targets[0].getName();
+          mirrorTarget = createSocketAddr(mirrorNode);
+          try {
+            mirrorSock = new Socket();
+            mirrorSock.connect(mirrorTarget, READ_TIMEOUT);
+            mirrorSock.setSoTimeout(READ_TIMEOUT);
+            mirrorOut = new DataOutputStream( 
+                        new BufferedOutputStream(mirrorSock.getOutputStream(),
+                                                 SMALL_HDR_BUFFER_SIZE));
+            mirrorIn = new DataInputStream( mirrorSock.getInputStream() );
+            //Copied from DFSClient.java!
+            mirrorOut.writeShort( DATA_TRANFER_VERSION );
+            mirrorOut.write( OP_WRITE_BLOCK );
+            mirrorOut.writeLong( block.getBlockId() );
+            mirrorOut.writeInt( targets.length - 1 );
+            for ( int i = 1; i < targets.length; i++ ) {
+              targets[i].write( mirrorOut );
             }
-          }
-              
-          //
-          // Process incoming data, copy to disk and
-          // maybe to network. First copy to the network before
-          // writing to local disk so that all datanodes might
-          // write to local disk in parallel.
-          //
-          boolean anotherChunk = len != 0;
-          byte buf[] = new byte[BUFFER_SIZE];
-              
-          while (anotherChunk) {
-            while (len > 0) {
-              int bytesRead = in.read(buf, 0, (int)Math.min(buf.length, len));
-              if (bytesRead < 0) {
-                throw new EOFException("EOF reading from "+s.toString());
-              }
-              if (bytesRead > 0) {
-                if (out2 != null) {
-                  try {
-                    out2.write(buf, 0, bytesRead);
-                  } catch (IOException out2e) {
-                    LOG.info("Exception writing to mirror " + mirrorNode 
-                             + "\n" + StringUtils.stringifyException(out2e));
-                    //
-                    // If stream-copy fails, continue 
-                    // writing to disk.  We shouldn't 
-                    // interrupt client write.
-                    //
-                    try {
-                      out2.close();
-                      in2.close();
-                    } catch (IOException out2close) {
-                    } finally {
-                      out2 = null;
-                      in2 = null;
-                    }
-                  }
-                }
-                try {
-                  out.write(buf, 0, bytesRead);
-                  myMetrics.wroteBytes(bytesRead);
-                } catch (IOException iex) {
-                  checkDiskError(iex);
-                  throw iex;
-                }
-                len -= bytesRead;
-              }
+            checksum.writeHeader( mirrorOut );
+            myMetrics.replicatedBlocks(1);
+          } catch (IOException ie) {
+            if (mirrorOut != null) {
+              LOG.info("Exception connecting to mirror " + mirrorNode 
+                       + "\n" + StringUtils.stringifyException(ie));
+              mirrorOut = null;
             }
-                
-            if (encodingType == RUNLENGTH_ENCODING) {
-              anotherChunk = false;
-            } else if (encodingType == CHUNKED_ENCODING) {
-              len = in.readLong();
-              if (out2 != null) {
-                try {
-                  out2.writeLong(len);
-                } catch (IOException ie) {
-                  LOG.info("Exception writing to mirror " + mirrorNode 
-                           + "\n" + StringUtils.stringifyException(ie));
-                  try {
-                    out2.close();
-                    in2.close();
-                  } catch (IOException ie2) {
-                    // NOTHING
-                  } finally {
-                    out2 = null;
-                    in2 = null;
-                  }
-                }
-              }
-              if (len == 0) {
-                anotherChunk = false;
-              }
+          }
+        }
+        
+        // XXX The following code is similar on both sides...
+        
+        int bytesPerChecksum = checksum.getBytesPerChecksum();
+        int checksumSize = checksum.getChecksumSize();
+        byte buf[] = new byte[ bytesPerChecksum + checksumSize ];
+        long blockLen = 0;
+        long lastOffset = 0;
+        long lastLen = 0;
+        int status = -1;
+        boolean headerWritten = false;
+        
+        while ( true ) {
+          // Read one data chunk in each loop.
+          
+          long offset = lastOffset + lastLen;
+          int len = (int) in.readInt();
+          if ( len < 0 || len > bytesPerChecksum ) {
+            LOG.warn( "Got wrong length during writeBlock(" +
+                      block + ") from " + s.getRemoteSocketAddress() +
+                      " at offset " + offset + ": " + len + 
+                      " expected <= " + bytesPerChecksum );
+            status = OP_STATUS_ERROR;
+            break;
+          }
+
+          in.readFully( buf, 0, len + checksumSize );
+          
+          if ( len > 0 && checksumSize > 0 ) {
+            /*
+             * Verification is not included in the initial design.
+             * For now, it at least catches some bugs. Later, we can 
+             * include this after showing that it does not affect 
+             * performance much.
+             */
+            checksum.update( buf, 0, len  );
+            
+            if ( ! checksum.compare( buf, len ) ) {
+              throw new IOException( "Unexpected checksum mismatch " +
+                                     "while writing " + block + 
+                                     " from " +
+                                     s.getRemoteSocketAddress() );
             }
+            
+            checksum.reset();
           }
-              
-          if (out2 != null) {
+
+          // First write to remote node before writing locally.
+          if (mirrorOut != null) {
             try {
-              out2.flush();
-              long complete = in2.readLong();
-              if (complete != WRITE_COMPLETE) {
-                LOG.info("Conflicting value for WRITE_COMPLETE: " + complete);
-              }
-              LocatedBlock newLB = new LocatedBlock();
-              newLB.readFields(in2);
-              in2.close();
-              out2.close();
-              DatanodeInfo mirrorsSoFar[] = newLB.getLocations();
-              for (int k = 0; k < mirrorsSoFar.length; k++) {
-                mirrors.add(mirrorsSoFar[k]);
-              }
-            } catch (IOException ie) {
-              LOG.info("Exception writing to mirror " + mirrorNode 
-                       + "\n" + StringUtils.stringifyException(ie));
-              try {
-                out2.close();
-                in2.close();
-              } catch (IOException ie2) {
-                // NOTHING
-              } finally {
-                out2 = null;
-                in2 = null;
-              }
+              mirrorOut.writeInt( len );
+              mirrorOut.write( buf, 0, len + checksumSize );
+            } catch (IOException ioe) {
+              LOG.info( "Exception writing to mirror " + mirrorNode + 
+                        "\n" + StringUtils.stringifyException(ioe) );
+              //
+              // If stream-copy fails, continue 
+              // writing to disk.  We shouldn't 
+              // interrupt client write.
+              //
+              mirrorOut = null;
             }
           }
-          if (out2 == null) {
-            LOG.info("Received block " + b + " from " + 
-                     s.getInetAddress());
-          } else {
-            LOG.info("Received block " + b + " from " + 
-                     s.getInetAddress() + 
-                     " and mirrored to " + mirrorTarget);
-          }
-        } finally {
+
           try {
-            out.close();
+            if ( !headerWritten ) { 
+              // First DATA_CHUNK. 
+              // Write the header even if checksumSize is 0.
+              checksumOut.writeShort( FSDataset.METADATA_VERSION );
+              checksum.writeHeader( checksumOut );
+              headerWritten = true;
+            }
+            
+            if ( len > 0 ) {
+              out.write( buf, 0, len );
+              // Write checksum
+              checksumOut.write( buf, len, checksumSize );
+              myMetrics.wroteBytes( len );
+            }
+            
           } catch (IOException iex) {
             checkDiskError(iex);
             throw iex;
           }
-        }
-        data.finalizeBlock(b);
-        myMetrics.wroteBlocks(1);
+          
+          if ( len == 0 ) {
+
+            // We already have one successful write here. Should we
+            // wait for response from next target? We will skip for now.
+
+            block.setNumBytes( blockLen );
             
-        // 
-        // Tell the namenode that we've received this block 
-        // in full, if we've been asked to.  This is done
-        // during NameNode-directed block transfers, but not
-        // client writes.
-        //
-        if (shouldReportBlock) {
-          synchronized (receivedBlockList) {
-            receivedBlockList.add(b);
+            //Does this fsync()?
+            data.finalizeBlock( block );
+            myMetrics.wroteBlocks(1);
+            
+            status = OP_STATUS_SUCCESS;
+            
+            break;
+          }
+          
+          if ( lastLen > 0 && lastLen != bytesPerChecksum ) {
+            LOG.warn( "Got wrong length during writeBlock(" +
+                      block + ") from " + s.getRemoteSocketAddress() +
+                      " : " + " got " + lastLen + " instead of " +
+                      bytesPerChecksum );
+            status = OP_STATUS_ERROR;
+            break;
+          }
+          
+          lastOffset = offset;
+          lastLen = len;
+          blockLen += len;
+        }
+        // done with reading the data.
+        
+        if ( status == OP_STATUS_SUCCESS ) {
+          /* Informing the name node could take a long long time!
+             Should we wait till namenode is informed before responding
+             with success to the client? For now we don't.
+          */
+          synchronized ( receivedBlockList ) {
+            receivedBlockList.add( block );
             receivedBlockList.notifyAll();
           }
+          
+          String msg = "Received block " + block + " from " + 
+                       s.getInetAddress();
+          
+          if ( mirrorOut != null ) {
+            //Wait for the remote reply
+            mirrorOut.flush();
+            byte result = OP_STATUS_ERROR; 
+            try {
+              result = mirrorIn.readByte();
+            } catch ( IOException ignored ) {}
+
+            msg += " and " +  (( result != OP_STATUS_SUCCESS ) ? 
+                                "failed to mirror to " : " mirrored to ") +
+                   mirrorTarget;
+            
+            mirrorOut = null;
+          }
+          
+          LOG.info(msg);
         }
             
+        if ( status >= 0 ) {
+          try {
+            reply.writeByte( status );
+            reply.flush();
+          } catch ( IOException ignored ) {}
+        }
+        
+      } finally {
+        try {
+          if ( out != null )
+            out.close();
+          if ( checksumOut != null )
+            checksumOut.close();
+          if ( mirrorSock != null )
+            mirrorSock.close();
+        } catch (IOException iex) {
+          shutdown();
+          throw iex;
+        }
+      }
+    }
+    
+    /**
+     * Reads the metadata and sends the data in one 'DATA_CHUNK'
+     * @param in
+     */
+    void readMetadata(DataInputStream in) throws IOException {
+      
+      Block block = new Block( in.readLong(), 0 );
+      InputStream checksumIn = null;
+      DataOutputStream out = null;
+      
+      try {
+        File blockFile = data.getBlockFile( block );
+        File checksumFile = FSDataset.getMetaFile( blockFile );
+        checksumIn = new FileInputStream(checksumFile);
+
+        long fileSize = checksumFile.length();
+        if (fileSize >= 1L<<31 || fileSize <= 0) {
+          throw new IOException("Unexpected size for checksumFile " +
+                                checksumFile);
+        }
+
+        byte [] buf = new byte[(int)fileSize];
+        FileUtil.readFully(checksumIn, buf, 0, buf.length);
+        
+        out = new DataOutputStream(s.getOutputStream());
+        
+        out.writeByte(OP_STATUS_SUCCESS);
+        out.writeInt(buf.length);
+        out.write(buf);
+        
+        //last DATA_CHUNK
+        out.writeInt(0);
+      } finally {
+        FileUtil.closeStream(checksumIn);
+      }
+    }
+  }
+
+  /** sendBlock() is used to read block and its metadata and stream
+   * the data to either a client or to another datanode.
+   * If argument targets is null, then it is assumed to be replying
+   * to a client request (OP_BLOCK_READ). Otherwise, we are replicating
+   * to another datanode.
+   * 
+   * returns total bytes reads, including crc.
+   */
+  long sendBlock(Socket sock, Block block,
+                 long startOffset, long length, DatanodeInfo targets[] )
+                 throws IOException {
+    // May be we should just use io.file.buffer.size.
+    DataOutputStream out = new DataOutputStream(
+                           new BufferedOutputStream(sock.getOutputStream(), 
+                                                    SMALL_HDR_BUFFER_SIZE));
+    DataInputStream in = null;
+    DataInputStream checksumIn = null;
+    long totalRead = 0;    
+
+
+    /* XXX This will affect inter datanode transfers during 
+     * a CRC upgrade. There should not be any replication
+     * during crc upgrade since we are in safe mode, right?
+     */    
+    boolean corruptChecksumOk = targets == null; 
+
+    try {
+      File blockFile = data.getBlockFile( block );
+      in = new DataInputStream( new FileInputStream( blockFile ) );
+
+      File checksumFile = FSDataset.getMetaFile( blockFile );
+      DataChecksum checksum = null;
+
+      if ( !corruptChecksumOk || checksumFile.exists() ) {
+        checksumIn = new DataInputStream( new FileInputStream(checksumFile) );
+          
+        //read and handle the common header here. For now just a version
+        short version = checksumIn.readShort();
+        if ( version != FSDataset.METADATA_VERSION ) {
+          LOG.warn( "Wrong version (" + version + 
+                    ") for metadata file for " + block + " ignoring ..." );
+        }
+        checksum = DataChecksum.newDataChecksum( checksumIn ) ;
+      } else {
+        LOG.warn( "Could not find metadata file for " + block );
+        // This only decides the buffer size. Use BUFFER_SIZE?
+        checksum = DataChecksum.newDataChecksum( DataChecksum.CHECKSUM_NULL,
+                                                 16*1024 );
+      }
+
+      int bytesPerChecksum = checksum.getBytesPerChecksum();
+      int checksumSize = checksum.getChecksumSize();
+
+      long endOffset = data.getLength( block );
+      if ( startOffset < 0 || startOffset > endOffset ||
+          (length + startOffset) > endOffset ) {
+        String msg = " Offset " + startOffset + " and length " + length + 
+                     " don't match block " + block +  " ( blockLen " + 
+                     endOffset + " )"; 
+        LOG.warn( "sendBlock() : " + msg );
+        if ( targets != null ) {
+          throw new IOException(msg);
+        } else {
+          out.writeShort( OP_STATUS_ERROR_INVALID );
+          return totalRead;
+        }
+      }
+
+      byte buf[] = new byte[ bytesPerChecksum + checksumSize ];
+      long offset = (startOffset - (startOffset % bytesPerChecksum));
+      if ( length >= 0 ) {
+        // Make sure endOffset points to end of a checksumed chunk. 
+        long tmpLen = startOffset + length + (startOffset - offset);
+        if ( tmpLen % bytesPerChecksum != 0 ) { 
+          tmpLen += ( bytesPerChecksum - tmpLen % bytesPerChecksum );
+        }
+        if ( tmpLen < endOffset ) {
+          endOffset = tmpLen;
+        }
+      }
+
+      // seek to the right offsets
+      if ( offset > 0 ) {
+        long checksumSkip = ( offset / bytesPerChecksum ) * checksumSize ;
+        /* XXX skip() could be very inefficent. Should be seek(). 
+         * at least skipFully
+         */
+        if ( in.skip( offset ) != offset || 
+            ( checksumSkip > 0 && 
+                checksumIn.skip( checksumSkip ) != checksumSkip ) ) {
+          throw new IOException( "Could not seek to right position while " +
+                                 "reading for " + block );
+        }
+      }
+      
+      if ( targets != null ) {
         //
-        // Tell client job is done, and reply with
-        // the new LocatedBlock.
+        // Header info
         //
-        reply.writeLong(WRITE_COMPLETE);
-        mirrors.add(curTarget);
-        LocatedBlock newLB = new LocatedBlock(b, mirrors.toArray(new DatanodeInfo[mirrors.size()]));
-        newLB.write(reply);
-      } finally {
-        reply.close();
+        out.writeShort( DATA_TRANFER_VERSION );
+        out.writeByte( OP_WRITE_BLOCK );
+        out.writeLong( block.getBlockId() );
+        out.writeInt(targets.length-1);
+        for (int i = 1; i < targets.length; i++) {
+          targets[i].write( out );
+        }
+      } else {
+        out.writeShort( OP_STATUS_SUCCESS );          
+      }
+
+      checksum.writeHeader( out );
+      
+      if ( targets == null ) {
+        out.writeLong( offset );
+      }
+      
+      while ( endOffset >= offset ) {
+        // Write one data chunk per loop.
+        int len = (int) Math.min( endOffset - offset, bytesPerChecksum );
+        if ( len > 0 ) {
+          in.readFully( buf, 0, len );
+          totalRead += len;
+          
+          if ( checksumSize > 0 && checksumIn != null ) {
+            try {
+              checksumIn.readFully( buf, len, checksumSize );
+              totalRead += checksumSize;
+            } catch ( IOException e ) {
+              LOG.warn( " Could not read checksum for data at offset " +
+                        offset + " for block " + block + " got : " + 
+                        StringUtils.stringifyException(e) );
+              FileUtil.closeStream( checksumIn );
+              checksumIn = null;
+              if ( corruptChecksumOk ) {
+                // Just fill the array with zeros.
+                Arrays.fill( buf, len, len + checksumSize, (byte)0 );
+              } else {
+                throw e;
+              }
+            }
+          }
+        }
+
+        out.writeInt( len );
+        out.write( buf, 0, len + checksumSize );
+        
+        if ( offset == endOffset ) {
+          out.flush();
+          // We are not waiting for response from target.
+          break;
+        }
+        offset += len;
       }
+    } finally {
+      FileUtil.closeStream( checksumIn );
+      FileUtil.closeStream( in );
+      FileUtil.closeStream( out );
     }
+    
+    return totalRead;
   }
 
   /**
@@ -1094,20 +1242,16 @@ public class DataNode implements FSConstants, Runnable {
    * sends a piece of data to another DataNode.
    */
   class DataTransfer implements Runnable {
-    InetSocketAddress curTarget;
     DatanodeInfo targets[];
     Block b;
-    byte buf[];
 
     /**
      * Connect to the first item in the target list.  Pass along the 
      * entire target list, the block, and the data.
      */
     public DataTransfer(DatanodeInfo targets[], Block b) throws IOException {
-      this.curTarget = createSocketAddr(targets[0].getName());
       this.targets = targets;
       this.b = b;
-      this.buf = new byte[BUFFER_SIZE];
     }
 
     /**
@@ -1115,46 +1259,23 @@ public class DataNode implements FSConstants, Runnable {
      */
     public void run() {
       xmitsInProgress++;
+      Socket sock = null;
+      
       try {
-        Socket s = new Socket();
-        s.connect(curTarget, READ_TIMEOUT);
-        s.setSoTimeout(READ_TIMEOUT);
-        DataOutputStream out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
-        try {
-          long filelen = data.getLength(b);
-          DataInputStream in = new DataInputStream(new BufferedInputStream(data.getBlockData(b)));
-          try {
-            //
-            // Header info
-            //
-            out.write(OP_WRITE_BLOCK);
-            out.writeBoolean(true);
-            b.write(out);
-            out.writeInt(targets.length);
-            for (int i = 0; i < targets.length; i++) {
-              targets[i].write(out);
-            }
-            out.write(RUNLENGTH_ENCODING);
-            out.writeLong(filelen);
+        InetSocketAddress curTarget = 
+          createSocketAddr(targets[0].getName());
+        sock = new Socket();  
+        sock.connect(curTarget, READ_TIMEOUT);
+        sock.setSoTimeout(READ_TIMEOUT);
+        sendBlock( sock, b, 0, -1, targets );
+        LOG.info( "Transmitted block " + b + " to " + curTarget );
 
-            //
-            // Write the data
-            //
-            while (filelen > 0) {
-              int bytesRead = in.read(buf, 0, (int) Math.min(filelen, buf.length));
-              out.write(buf, 0, bytesRead);
-              filelen -= bytesRead;
-            }
-          } finally {
-            in.close();
-          }
-        } finally {
-          out.close();
-        }
-        LOG.info("Transmitted block " + b + " to " + curTarget);
-      } catch (IOException ie) {
-        LOG.warn("Failed to transfer "+b+" to "+curTarget, ie);
+      } catch ( IOException ie ) {
+        LOG.warn( "Failed to transfer " + b + " to " + 
+                  targets[0].getName() + " got " + 
+                  StringUtils.stringifyException( ie ) );
       } finally {
+        FileUtil.closeSocket(sock);
         xmitsInProgress--;
       }
     }

+ 15 - 2
src/java/org/apache/hadoop/dfs/DatanodeProtocol.java

@@ -31,9 +31,9 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  **********************************************************************/
 interface DatanodeProtocol extends VersionedProtocol {
   /*
-   * 7: processUpgradeCommand() added;
+   * 8: blockCrcUpgradeGetBlockLocations() added;
    */
-  public static final long versionID = 7L;
+  public static final long versionID = 8L;
   
   // error code
   final static int NOTIFY = 0;
@@ -116,4 +116,17 @@ interface DatanodeProtocol extends VersionedProtocol {
    * @return a reply in the form of an upgrade command
    */
   UpgradeCommand processUpgradeCommand(UpgradeCommand comm) throws IOException;
+  
+  /**
+   * Get locations for a given block. This is used only during 
+   * Block Level CRC upgrades (HADOOP-1134), when a datanode node
+   * misses the cluster wide distributed upgrade. It uses the same
+   * BlockCrcInfo class, that is also used during distributed upgrade
+   * 
+   * @param block 
+   * @return BlockCrcInfo that contains the block locations.
+   * @throws IOException
+   */
+  public BlockCrcInfo blockCrcUpgradeGetBlockLocations(Block block)
+                                                      throws IOException;  
 }

+ 211 - 308
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -33,367 +33,233 @@ import org.apache.hadoop.util.*;
  * DistributedFileSystem.
  *
  *****************************************************************/
-public class DistributedFileSystem extends ChecksumFileSystem {
-  private static class RawDistributedFileSystem extends FileSystem {
-    private Path workingDir =
-      new Path("/user", System.getProperty("user.name")); 
-    private URI uri;
-    private FileSystem localFs;
+public class DistributedFileSystem extends FileSystem {
+  private Path workingDir =
+    new Path("/user", System.getProperty("user.name")); 
+  private URI uri;
+  private FileSystem localFs;
 
-    DFSClient dfs;
+  DFSClient dfs;
 
-    public RawDistributedFileSystem() {
-    }
-
-
-    /** @deprecated */
-    public RawDistributedFileSystem(InetSocketAddress namenode,
-                                    Configuration conf) throws IOException {
-      initialize(URI.create("hdfs://"+
-                            namenode.getHostName()+":"+
-                            namenode.getPort()),
-                 conf);
-    }
+  public DistributedFileSystem() {
+  }
 
-    /** @deprecated */
-    public String getName() { return uri.getAuthority(); }
 
-    public URI getUri() { return uri; }
+  /** @deprecated */
+  public DistributedFileSystem(InetSocketAddress namenode,
+    Configuration conf) throws IOException {
+    initialize(URI.create("hdfs://"+
+                          namenode.getHostName()+":"+
+                          namenode.getPort()),
+                          conf);
+  }
 
-    public void initialize(URI uri, Configuration conf) throws IOException {
-      setConf(conf);
-      String host = uri.getHost();
-      int port = uri.getPort();
-      this.dfs = new DFSClient(new InetSocketAddress(host, port), conf);
-      this.uri = URI.create("hdfs://"+host+":"+port);
-      this.localFs = getLocal(conf);
-    }
+  /** @deprecated */
+  public String getName() { return uri.getAuthority(); }
 
-    public Path getWorkingDirectory() {
-      return workingDir;
-    }
-    
-    public long getDefaultBlockSize() {
-      return dfs.getDefaultBlockSize();
-    }
-    
-    public short getDefaultReplication() {
-      return dfs.getDefaultReplication();
-    }
-    
-    private Path makeAbsolute(Path f) {
-      if (f.isAbsolute()) {
-        return f;
-      } else {
-        return new Path(workingDir, f);
-      }
-    }
-    
-    public void setWorkingDirectory(Path dir) {
-      Path result = makeAbsolute(dir);
-      if (!FSNamesystem.isValidName(result.toString())) {
-        throw new IllegalArgumentException("Invalid DFS directory name " + 
-                                           result);
-      }
-      workingDir = makeAbsolute(dir);
-    }
-    
-    /**
-     * @deprecated use {@link #getPathName(Path)} instead.
-     */
-    private UTF8 getPath(Path file) {
-      return new UTF8(getPathName(file));
-    }
+  public URI getUri() { return uri; }
 
-    private String getPathName(Path file) {
-      checkPath(file);
-      String result = makeAbsolute(file).toUri().getPath();
-      if (!FSNamesystem.isValidName(result)) {
-        throw new IllegalArgumentException("Pathname " + result + " from " +
-                                           file +
-                                           " is not a valid DFS filename.");
-      }
-      return result;
-    }
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    setConf(conf);
+    String host = uri.getHost();
+    int port = uri.getPort();
+    this.dfs = new DFSClient(new InetSocketAddress(host, port), conf);
+    this.uri = URI.create("hdfs://"+host+":"+port);
+    this.localFs = getLocal(conf);
+  }
 
-    public String[][] getFileCacheHints(Path f, long start, long len) throws IOException {
-      return dfs.getHints(getPathName(f), start, len);
-    }
+  public Path getWorkingDirectory() {
+    return workingDir;
+  }
 
-    public FSDataInputStream open(Path f, int bufferSize) throws IOException {
-      return new DFSClient.DFSDataInputStream(dfs.open(getPath(f),bufferSize));
-    }
+  public long getDefaultBlockSize() {
+    return dfs.getDefaultBlockSize();
+  }
 
-    public FSDataOutputStream create(Path f, boolean overwrite,
-                                     int bufferSize, short replication, long blockSize,
-                                     Progressable progress) throws IOException {
-      if (exists(f) && !overwrite) {
-        throw new IOException("File already exists:"+f);
-      }
-      Path parent = f.getParent();
-      if (parent != null && !mkdirs(parent)) {
-        throw new IOException("Mkdirs failed to create " + parent);
-      }
-      
-      return new FSDataOutputStream( dfs.create(
-          getPath(f), overwrite, replication, blockSize, progress, bufferSize));
-    }
-    
-    public boolean setReplication(Path src, 
-                                  short replication
-                                  ) throws IOException {
-      return dfs.setReplication(getPath(src), replication);
-    }
-    
-    /**
-     * Rename files/dirs
-     */
-    public boolean rename(Path src, Path dst) throws IOException {
-      return dfs.rename(getPath(src), getPath(dst));
-    }
+  public short getDefaultReplication() {
+    return dfs.getDefaultReplication();
+  }
 
-    /**
-     * Get rid of Path f, whether a true file or dir.
-     */
-    public boolean delete(Path f) throws IOException {
-      return dfs.delete(getPath(f));
+  private Path makeAbsolute(Path f) {
+    if (f.isAbsolute()) {
+      return f;
+    } else {
+      return new Path(workingDir, f);
     }
+  }
 
-    public boolean exists(Path f) throws IOException {
-      return dfs.exists(getPath(f));
+  public void setWorkingDirectory(Path dir) {
+    Path result = makeAbsolute(dir);
+    if (!FSNamesystem.isValidName(result.toString())) {
+      throw new IllegalArgumentException("Invalid DFS directory name " + 
+                                         result);
     }
+    workingDir = makeAbsolute(dir);
+  }
 
-    public long getContentLength(Path f) throws IOException {
-      if (f instanceof DfsPath) {
-        return ((DfsPath)f).getContentsLength();
-      }
-
-      DFSFileInfo info[] = dfs.listPaths(getPath(f));
-      return (info == null) ? 0 : info[0].getLen();
-    }
+  /**
+   * @deprecated use {@link #getPathName(Path)} instead.
+   */
+  private UTF8 getPath(Path file) {
+    return new UTF8(getPathName(file));
+  }
 
-    public Path[] listPaths(Path f) throws IOException {
-      DFSFileInfo info[] = dfs.listPaths(getPath(f));
-      if (info == null) {
-        return new Path[0];
-      } else {
-        Path results[] = new DfsPath[info.length];
-        for (int i = 0; i < info.length; i++) {
-          results[i] = new DfsPath(info[i], this);
-        }
-        return results;
-      }
+  private String getPathName(Path file) {
+    checkPath(file);
+    String result = makeAbsolute(file).toUri().getPath();
+    if (!FSNamesystem.isValidName(result)) {
+      throw new IllegalArgumentException("Pathname " + result + " from " +
+                                         file+" is not a valid DFS filename.");
     }
+    return result;
+  }
 
-    public boolean mkdirs(Path f) throws IOException {
-      return dfs.mkdirs(getPath(f));
-    }
+  public String[][] getFileCacheHints(Path f, long start, long len) throws IOException {
+    return dfs.getHints(getPathName(f), start, len);
+  }
 
-    /** @deprecated */ @Deprecated
-      public void lock(Path f, boolean shared) throws IOException {
-      dfs.lock(getPath(f), !shared);
-    }
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    return new DFSClient.DFSDataInputStream(dfs.open(getPath(f),bufferSize));
+  }
 
-    /** @deprecated */ @Deprecated
-      public void release(Path f) throws IOException {
-      dfs.release(getPath(f));
+  public FSDataOutputStream create(Path f, boolean overwrite,
+    int bufferSize, short replication, long blockSize,
+    Progressable progress) throws IOException {
+    if (exists(f) && !overwrite) {
+      throw new IOException("File already exists:"+f);
     }
-
-    @Override
-    public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
-      throws IOException {
-      FileUtil.copy(localFs, src, this, dst, delSrc, getConf());
+    Path parent = f.getParent();
+    if (parent != null && !mkdirs(parent)) {
+      throw new IOException("Mkdirs failed to create " + parent);
     }
 
-    @Override
-    public void copyToLocalFile(boolean delSrc, Path src, Path dst)
-      throws IOException {
-      FileUtil.copy(this, src, localFs, dst, delSrc, getConf());
-    }
-
-    public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
-      throws IOException {
-      return tmpLocalFile;
-    }
+    return new FSDataOutputStream( dfs.create(getPath(f), overwrite, 
+                                              replication, blockSize, 
+                                              progress, bufferSize) );
+  }
 
-    /**
-     * Move completed local data to DFS destination
-     */
-    public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
-      throws IOException {
-      moveFromLocalFile(tmpLocalFile, fsOutputFile);
-    }
+  public boolean setReplication(Path src, 
+                                short replication
+                               ) throws IOException {
+    return dfs.setReplication(getPath(src), replication);
+  }
 
-    public void close() throws IOException {
-      super.close();
-      dfs.close();
-    }
+  /**
+   * Rename files/dirs
+   */
+  public boolean rename(Path src, Path dst) throws IOException {
+    return dfs.rename(getPath(src), getPath(dst));
+  }
 
-    public String toString() {
-      return "DFS[" + dfs + "]";
-    }
+  /**
+   * Get rid of Path f, whether a true file or dir.
+   */
+  public boolean delete(Path f) throws IOException {
+    return dfs.delete(getPath(f));
+  }
 
-    DFSClient getClient() {
-      return dfs;
-    }        
-    /** Return the total raw capacity of the filesystem, disregarding
-     * replication .*/
-    public long getRawCapacity() throws IOException{
-      return dfs.totalRawCapacity();
-    }
+  public boolean exists(Path f) throws IOException {
+    return dfs.exists(getPath(f));
+  }
 
-    /** Return the total raw used space in the filesystem, disregarding
-     * replication .*/
-    public long getRawUsed() throws IOException{
-      return dfs.totalRawUsed();
+  public long getContentLength(Path f) throws IOException {
+    if (f instanceof DfsPath) {
+      return ((DfsPath)f).getContentsLength();
     }
 
-    /** Return statistics for each datanode. */
-    public DatanodeInfo[] getDataNodeStats() throws IOException {
-      return dfs.datanodeReport();
-    }
-    
-    /**
-     * Enter, leave or get safe mode.
-     *  
-     * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
-     */
-    public boolean setSafeMode(FSConstants.SafeModeAction action) 
-      throws IOException {
-      return dfs.setSafeMode(action);
-    }
+    DFSFileInfo info[] = dfs.listPaths(getPath(f));
+    return (info == null) ? 0 : info[0].getLen();
+  }
 
-    /*
-     * Refreshes the list of hosts and excluded hosts from the configured 
-     * files.  
-     */
-    public void refreshNodes() throws IOException {
-      dfs.refreshNodes();
+  public Path[] listPaths(Path f) throws IOException {
+    DFSFileInfo info[] = dfs.listPaths(getPath(f));
+    if (info == null) {
+      return new Path[0];
+    } else {
+      Path results[] = new DfsPath[info.length];
+      for (int i = 0; i < info.length; i++) {
+        results[i] = new DfsPath(info[i], this);
+      }
+      return results;
     }
+  }
 
-    /**
-     * Finalize previously upgraded files system state.
-     * @throws IOException
-     */
-    public void finalizeUpgrade() throws IOException {
-      dfs.finalizeUpgrade();
-    }
+  public boolean mkdirs(Path f) throws IOException {
+    return dfs.mkdirs(getPath(f));
+  }
 
-    public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
-                                                          ) throws IOException {
-      return dfs.distributedUpgradeProgress(action);
-    }
+  /** @deprecated */ @Deprecated
+  public void lock(Path f, boolean shared) throws IOException {
+    dfs.lock(getPath(f), !shared);
+  }
 
-    /*
-     * Requests the namenode to dump data strcutures into specified 
-     * file.
-     */
-    public void metaSave(String pathname) throws IOException {
-      dfs.metaSave(pathname);
-    }
+  /** @deprecated */ @Deprecated
+  public void release(Path f) throws IOException {
+    dfs.release(getPath(f));
+  }
 
-    /**
-     * We need to find the blocks that didn't match.  Likely only one 
-     * is corrupt but we will report both to the namenode.  In the future,
-     * we can consider figuring out exactly which block is corrupt.
-     */
-    public boolean reportChecksumFailure(Path f, 
-                                         FSDataInputStream in, long inPos, 
-                                         FSDataInputStream sums, long sumsPos) {
-      
-      LocatedBlock lblocks[] = new LocatedBlock[2];
-
-      try {
-        // Find block in data stream.
-        DFSClient.DFSDataInputStream dfsIn = (DFSClient.DFSDataInputStream) in;
-        Block dataBlock = dfsIn.getCurrentBlock();
-        if (dataBlock == null) {
-          throw new IOException("Error: Current block in data stream is null! ");
-        }
-        DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
-        lblocks[0] = new LocatedBlock(dataBlock, dataNode);
-        LOG.info("Found checksum error in data stream at block=" + dataBlock.getBlockName() + 
-                 " on datanode=" + dataNode[0].getName());
-
-        // Find block in checksum stream
-        DFSClient.DFSDataInputStream dfsSums = (DFSClient.DFSDataInputStream) sums;
-        Block sumsBlock = dfsSums.getCurrentBlock();
-        if (sumsBlock == null) {
-          throw new IOException("Error: Current block in checksum stream is null! ");
-        }
-        DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
-        lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
-        LOG.info("Found checksum error in checksum stream at block=" + sumsBlock.getBlockName() + 
-                 " on datanode=" + sumsNode[0].getName());
-
-        // Ask client to delete blocks.
-        dfs.reportBadBlocks(lblocks);
-
-      } catch (IOException ie) {
-        LOG.info("Found corruption while reading "
-                 + f.toString() 
-                 + ".  Error repairing corrupt blocks.  Bad blocks remain. " 
-                 + StringUtils.stringifyException(ie));
-      }
+  @Override
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
+  throws IOException {
+    FileUtil.copy(localFs, src, this, dst, delSrc, getConf());
+  }
 
-      return true;
-    }
+  @Override
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst)
+  throws IOException {
+    FileUtil.copy(this, src, localFs, dst, delSrc, getConf());
+  }
 
-    /**
-     * Returns the stat information about the file.
-     */
-    public FileStatus getFileStatus(Path f) throws IOException {
-      if (f instanceof DfsPath) {
-        DfsPath p = (DfsPath) f;
-        return p.info;
-      }
-      else {
-        DFSFileInfo p = dfs.getFileInfo(getPath(f));
-        return p;
-      }
-    }
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+  throws IOException {
+    return tmpLocalFile;
   }
 
-  public DistributedFileSystem() {
-    super(new RawDistributedFileSystem());
+  /**
+   * Move completed local data to DFS destination
+   */
+  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
+  throws IOException {
+    moveFromLocalFile(tmpLocalFile, fsOutputFile);
   }
 
-  /** @deprecated */
-  public DistributedFileSystem(InetSocketAddress namenode,
-                               Configuration conf) throws IOException {
-    super(new RawDistributedFileSystem(namenode, conf));
+  public void close() throws IOException {
+    super.close();
+    dfs.close();
   }
 
-  @Override
-  public long getContentLength(Path f) throws IOException {
-    return fs.getContentLength(f);
+  public String toString() {
+    return "DFS[" + dfs + "]";
   }
 
+  DFSClient getClient() {
+    return dfs;
+  }        
   /** Return the total raw capacity of the filesystem, disregarding
    * replication .*/
   public long getRawCapacity() throws IOException{
-    return ((RawDistributedFileSystem)fs).getRawCapacity();
+    return dfs.totalRawCapacity();
   }
 
   /** Return the total raw used space in the filesystem, disregarding
    * replication .*/
   public long getRawUsed() throws IOException{
-    return ((RawDistributedFileSystem)fs).getRawUsed();
+    return dfs.totalRawUsed();
   }
 
   /** Return statistics for each datanode. */
   public DatanodeInfo[] getDataNodeStats() throws IOException {
-    return ((RawDistributedFileSystem)fs).getDataNodeStats();
+    return dfs.datanodeReport();
   }
-    
+
   /**
    * Enter, leave or get safe mode.
    *  
    * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
    */
   public boolean setSafeMode(FSConstants.SafeModeAction action) 
-    throws IOException {
-    return ((RawDistributedFileSystem)fs).setSafeMode(action);
+  throws IOException {
+    return dfs.setSafeMode(action);
   }
 
   /*
@@ -401,26 +267,28 @@ public class DistributedFileSystem extends ChecksumFileSystem {
    * files.  
    */
   public void refreshNodes() throws IOException {
-    ((RawDistributedFileSystem)fs).refreshNodes();
+    dfs.refreshNodes();
   }
 
   /**
    * Finalize previously upgraded files system state.
+   * @throws IOException
    */
   public void finalizeUpgrade() throws IOException {
-    ((RawDistributedFileSystem)fs).finalizeUpgrade();
+    dfs.finalizeUpgrade();
   }
 
   public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action
-                                                        ) throws IOException {
-    return ((RawDistributedFileSystem)fs).distributedUpgradeProgress(action);
+  ) throws IOException {
+    return dfs.distributedUpgradeProgress(action);
   }
 
   /*
-   * Dumps dfs data structures into specified file.
+   * Requests the namenode to dump data strcutures into specified 
+   * file.
    */
   public void metaSave(String pathname) throws IOException {
-    ((RawDistributedFileSystem)fs).metaSave(pathname);
+    dfs.metaSave(pathname);
   }
 
   /**
@@ -429,17 +297,52 @@ public class DistributedFileSystem extends ChecksumFileSystem {
    * we can consider figuring out exactly which block is corrupt.
    */
   public boolean reportChecksumFailure(Path f, 
-                                       FSDataInputStream in, long inPos, 
-                                       FSDataInputStream sums, long sumsPos) {
-    return ((RawDistributedFileSystem)fs).reportChecksumFailure(
-                                                                f, in, inPos, sums, sumsPos);
+    FSDataInputStream in, long inPos, 
+    FSDataInputStream sums, long sumsPos) {
+
+    LocatedBlock lblocks[] = new LocatedBlock[2];
+
+    // Find block in data stream.
+    DFSClient.DFSDataInputStream dfsIn = (DFSClient.DFSDataInputStream) in;
+    Block dataBlock = dfsIn.getCurrentBlock();
+    if (dataBlock == null) {
+      LOG.error("Error: Current block in data stream is null! ");
+      return false;
+    }
+    DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
+    lblocks[0] = new LocatedBlock(dataBlock, dataNode);
+    LOG.info("Found checksum error in data stream at block=" + dataBlock.getBlockName() + 
+             " on datanode=" + dataNode[0].getName());
+
+    // Find block in checksum stream
+    DFSClient.DFSDataInputStream dfsSums = (DFSClient.DFSDataInputStream) sums;
+    Block sumsBlock = dfsSums.getCurrentBlock();
+    if (sumsBlock == null) {
+      LOG.error("Error: Current block in checksum stream is null! ");
+      return false;
+    }
+    DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
+    lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
+    LOG.info("Found checksum error in checksum stream at block=" + sumsBlock.getBlockName() + 
+             " on datanode=" + sumsNode[0].getName());
+
+    // Ask client to delete blocks.
+    dfs.reportChecksumFailure(f.toString(), lblocks);
+
+    return true;
   }
 
   /**
    * Returns the stat information about the file.
    */
-  @Override
   public FileStatus getFileStatus(Path f) throws IOException {
-    return ((RawDistributedFileSystem)fs).getFileStatus(f);
+    if (f instanceof DfsPath) {
+      DfsPath p = (DfsPath) f;
+      return p.info;
+    }
+    else {
+      DFSFileInfo p = dfs.getFileInfo(getPath(f));
+      return p;
+    }
   }
 }

+ 18 - 9
src/java/org/apache/hadoop/dfs/FSConstants.java

@@ -86,12 +86,20 @@ public interface FSConstants {
   // Processed at datanode stream-handler
   public static final byte OP_WRITE_BLOCK = (byte) 80;
   public static final byte OP_READ_BLOCK = (byte) 81;
-  public static final byte OP_READSKIP_BLOCK = (byte) 82;
-  public static final byte OP_READ_RANGE_BLOCK = (byte) 83;
-
-  // Encoding types
-  public static final byte RUNLENGTH_ENCODING = 0;
-  public static final byte CHUNKED_ENCODING = 1;
+  public static final byte OP_READ_METADATA = (byte) 82;
+  
+  public static final int OP_STATUS_SUCCESS = 0;  
+  public static final int OP_STATUS_ERROR = 1;  
+  public static final int OP_STATUS_ERROR_CHECKSUM = 2;  
+  public static final int OP_STATUS_ERROR_INVALID = 3;  
+  public static final int OP_STATUS_ERROR_EXISTS = 4;  
+
+  
+  /** Version for data transfers between clients and datanodes
+   * This should change when serialization of DatanodeInfo, not just
+   * when protocol changes. It is not very obvious. 
+   */
+  public static final int DATA_TRANFER_VERSION = 5; //Should it be 1?
 
   // Return codes for file create
   public static final int OPERATION_FAILED = 0;
@@ -115,8 +123,9 @@ public interface FSConstants {
   public static int MAX_PATH_LENGTH = 8000;
   public static int MAX_PATH_DEPTH = 1000;
     
-  //TODO mb@media-style.com: should be conf injected?
   public static final int BUFFER_SIZE = new Configuration().getInt("io.file.buffer.size", 4096);
+  //TODO mb@media-style.com: should be conf injected?
+  public static final long DEFAULT_BLOCK_SIZE = 64 * 1024 * 1024;
 
   // SafeMode actions
   public enum SafeModeAction{ SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; }
@@ -149,7 +158,7 @@ public interface FSConstants {
   // Version is reflected in the data storage file.
   // Versions are negative.
   // Decrement LAYOUT_VERSION to define a new version.
-  public static final int LAYOUT_VERSION = -6;
+  public static final int LAYOUT_VERSION = -7;
   // Current version: 
-  // Dustributed upgrade is introduced.
+  // Block Level CRCs added.
 }

+ 52 - 11
src/java/org/apache/hadoop/dfs/FSDataset.java

@@ -87,7 +87,14 @@ class FSDataset implements FSConstants {
                           boolean resetIdx) throws IOException {
       if (numBlocks < maxBlocksPerDir) {
         File dest = new File(dir, b.getBlockName());
-        src.renameTo(dest);
+        File metaData = getMetaFile( src );
+        if ( ! metaData.renameTo( getMetaFile(dest) ) ||
+            ! src.renameTo( dest ) ) {
+          throw new IOException( "could not move files for " + b +
+                                 " from tmp to " + 
+                                 dest.getAbsolutePath() );
+        }
+
         numBlocks += 1;
         return dest;
       }
@@ -155,6 +162,7 @@ class FSDataset implements FSConstants {
 
       File blockFiles[] = dir.listFiles();
       for (int i = 0; i < blockFiles.length; i++) {
+        //We are not enforcing presense of metadata file
         if (Block.isBlockFilename(blockFiles[i])) {
           volumeMap.put(new Block(blockFiles[i], blockFiles[i].length()), volume);
         }
@@ -262,6 +270,8 @@ class FSDataset implements FSConstants {
     
     FSVolume(File currentDir, Configuration conf) throws IOException {
       this.reserved = conf.getLong("dfs.datanode.du.reserved", 0);
+      // add block size to the configured reserved space
+      this.reserved += conf.getLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
       this.usableDiskPct = conf.getFloat("dfs.datanode.du.pct",
                                          (float) USABLE_DISK_PCT_DEFAULT);
       File parent = currentDir.getParentFile();
@@ -418,6 +428,14 @@ class FSDataset implements FSConstants {
   //
   //////////////////////////////////////////////////////
 
+  //Find better place?
+  public static final String METADATA_EXTENSION = ".meta";
+  public static final short METADATA_VERSION = 1;
+    
+  public static File getMetaFile( File f ) {
+    return new File( f.getAbsolutePath() + METADATA_EXTENSION );
+  }
+    
   FSVolumeSet volumes;
   private HashMap<Block,File> ongoingCreates = new HashMap<Block,File>();
   private int maxBlocksPerDir = 0;
@@ -467,20 +485,29 @@ class FSDataset implements FSConstants {
   }
 
   /**
-   * Get a stream of data from the indicated block.
+   * Get File name for a given block.
    */
-  public synchronized InputStream getBlockData(Block b) throws IOException {
+  public synchronized File getBlockFile(Block b) throws IOException {
     if (!isValidBlock(b)) {
       throw new IOException("Block " + b + " is not valid.");
     }
-    // File should be opened with the lock.
-    return new FileInputStream(getFile(b));
+    return getFile(b);
   }
 
+  static class BlockWriteStreams {
+    OutputStream dataOut;
+    OutputStream checksumOut;
+    
+    BlockWriteStreams( File f ) throws IOException {
+      dataOut = new FileOutputStream( f );
+      checksumOut = new FileOutputStream( getMetaFile( f ) );
+    }
+  }
+  
   /**
    * Start writing to a block file
    */
-  public OutputStream writeToBlock(Block b) throws IOException {
+  public BlockWriteStreams writeToBlock(Block b) throws IOException {
     //
     // Make sure the block isn't a valid one - we're still creating it!
     //
@@ -515,7 +542,7 @@ class FSDataset implements FSConstants {
       synchronized (volumes) {
         v = volumes.getNextVolume(blockSize);
         // create temporary file to hold block in the designated volume
-        f = v.createTmpFile(b);
+        f = createTmpFile(v, b);
       }
       ongoingCreates.put(b, f);
       volumeMap.put(b, v);
@@ -526,9 +553,23 @@ class FSDataset implements FSConstants {
     // REMIND - mjc - make this a filter stream that enforces a max
     // block size, so clients can't go crazy
     //
-    return new FileOutputStream(f);
+    return new BlockWriteStreams( f );
   }
 
+  File createTmpFile( FSVolume vol, Block blk ) throws IOException {
+    if ( vol == null ) {
+      synchronized ( this ) {
+        vol = volumeMap.get( blk );
+        if ( vol == null ) {
+          throw new IOException("Could not find volume for block " + blk);
+        }
+      }
+    }
+    synchronized ( volumes ) {
+      return vol.createTmpFile(blk);
+    }
+  }
+  
   //
   // REMIND - mjc - eventually we should have a timeout system
   // in place to clean up block files left by abandoned clients.
@@ -545,8 +586,6 @@ class FSDataset implements FSConstants {
     if (f == null || !f.exists()) {
       throw new IOException("No temporary file " + f + " for block " + b);
     }
-    long finalLen = f.length();
-    b.setNumBytes(finalLen);
     FSVolume v = volumeMap.get(b);
         
     File dest = null;
@@ -575,6 +614,7 @@ class FSDataset implements FSConstants {
    * Check whether the given block is a valid one.
    */
   public boolean isValidBlock(Block b) {
+    //Should we check for metadata file too?
     File f = getFile(b);
     return (f!= null && f.exists());
   }
@@ -619,7 +659,8 @@ class FSDataset implements FSConstants {
         blockMap.remove(invalidBlks[i]);
         volumeMap.remove(invalidBlks[i]);
       }
-      if (!f.delete()) {
+      File metaFile = getMetaFile( f );
+      if ( !f.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
         DataNode.LOG.warn("Unexpected error trying to delete block "
                           + invalidBlks[i] + " at file " + f);
         error = true;

+ 7 - 0
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -113,6 +113,13 @@ class FSDirectory implements FSConstants {
     String getLocalName() {
       return name;
     }
+    
+    String getAbsoluteName() {
+      // recursively constructs the absolute path.
+      // Any escaping of name required?
+      return ((parent != null) ? 
+              (parent.getAbsoluteName() + Path.SEPARATOR): "") + name;
+    }
 
     /**
      * Get file blocks 

+ 116 - 3
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -416,6 +416,109 @@ class FSNamesystem implements FSConstants {
                               curReplicasDelta, expectedReplicasDelta);
   }
 
+  /**
+   * Used only during DFS upgrade for block level CRCs (HADOOP-1134).
+   * This returns information for a given blocks that includes:
+   * <li> full path name for the file that contains the block.
+   * <li> offset of first byte of the block.
+   * <li> file length and length of the block.
+   * <li> all block locations for the crc file (".file.crc").
+   * <li> replication for crc file.
+   * When replicas is true, it includes replicas of the block.
+   */
+  public synchronized BlockCrcInfo blockCrcInfo(Block block, 
+                                                boolean replicas) {
+    BlockCrcInfo crcInfo = new BlockCrcInfo();
+    crcInfo.status = BlockCrcInfo.STATUS_ERROR;
+    
+    FSDirectory.INode fileINode = blocksMap.getINode(block);
+    if ( fileINode == null || fileINode.isDir() ) {
+      // Most probably reason is that this block does not exist
+      if (blocksMap.getStoredBlock(block) == null) {
+        crcInfo.status = BlockCrcInfo.STATUS_UNKNOWN_BLOCK;
+      } else {
+        LOG.warn("getBlockCrcInfo(): Could not find file for " + block);
+      }
+      return crcInfo;
+    }
+
+    crcInfo.fileName = fileINode.getAbsoluteName();
+    
+    // Find the offset and length for this block.
+    Block[] fileBlocks = fileINode.getBlocks();
+    crcInfo.blockLen = -1;
+    if ( fileBlocks != null ) {
+      for ( Block b:fileBlocks ) {
+        if ( block.equals(b) ) {
+          crcInfo.blockLen = b.getNumBytes();
+        }
+        if ( crcInfo.blockLen < 0 ) {
+          crcInfo.startOffset += b.getNumBytes();
+        }
+        crcInfo.fileSize += b.getNumBytes();
+      }
+    }
+
+    if ( crcInfo.blockLen <= 0 ) {
+      LOG.warn("blockCrcInfo(): " + block + 
+               " could not be found in blocks for " + crcInfo.fileName);
+      return crcInfo;
+    }
+    
+    String fileName = fileINode.getLocalName();    
+    if ( fileName.startsWith(".") && fileName.endsWith(".crc") ) {
+      crcInfo.status = BlockCrcInfo.STATUS_CRC_BLOCK;
+      return crcInfo;
+    }
+
+    if (replicas) {
+      // include block replica locations, instead of crcBlocks
+      crcInfo.blockLocationsIncluded = true;
+      
+      DatanodeInfo[] dnInfo = new DatanodeInfo[blocksMap.numNodes(block)];
+      Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
+      for (int i=0; it != null && it.hasNext(); i++ ) {
+        dnInfo[i] = new DatanodeInfo(it.next());
+      }
+      crcInfo.blockLocations = new LocatedBlock(block, dnInfo, 
+                                                crcInfo.startOffset);
+    } else {
+
+      //Find CRC file
+      String crcName = "." + fileName + ".crc";
+      FSDirectory.INode crcINode = fileINode.getParent().getChild(crcName);
+
+      if ( crcINode == null ) {
+        // Should we log this?
+        crcInfo.status = BlockCrcInfo.STATUS_NO_CRC_DATA;
+        return crcInfo;
+      }
+
+      Block[] blocks = crcINode.getBlocks();
+      if ( blocks == null )  {
+        LOG.warn("getBlockCrcInfo(): could not find blocks for crc file for " +
+                 crcInfo.fileName);
+        return crcInfo;
+      }
+
+      crcInfo.crcBlocks = new LocatedBlock[ blocks.length ];
+      for (int i=0; i<blocks.length; i++) {
+        DatanodeInfo[] dnArr = new DatanodeInfo[ blocksMap.numNodes(blocks[i]) ];
+        int idx = 0;
+        for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(blocks[i]); 
+        it.hasNext();) { 
+          dnArr[ idx++ ] = it.next();
+        }
+        crcInfo.crcBlocks[i] = new LocatedBlock(blocks[i], dnArr);
+      }
+
+      crcInfo.crcReplication = crcINode.getReplication();
+    }
+    
+    crcInfo.status = BlockCrcInfo.STATUS_DATA_BLOCK;
+    return crcInfo;
+  }
+  
   /////////////////////////////////////////////////////////
   //
   // These methods are called by HadoopFS clients
@@ -1120,18 +1223,28 @@ class FSNamesystem implements FSConstants {
    * invalidate some blocks that make up the file.
    */
   public boolean delete(String src) throws IOException {
-    boolean status = deleteInternal(src);
+    boolean status = deleteInternal(src, true);
     getEditLog().logSync();
     return status;
   }
 
+  /**
+   * An internal delete function that does not enforce safe mode
+   */
+  boolean deleteInSafeMode(String src) throws IOException {
+    boolean status = deleteInternal(src, false);
+    getEditLog().logSync();
+    return status;
+  }
   /**
    * Remove the indicated filename from the namespace.  This may
    * invalidate some blocks that make up the file.
    */
-  private synchronized boolean deleteInternal(String src) throws IOException {
+  private synchronized boolean deleteInternal(String src, 
+                                              boolean enforceSafeMode) 
+                                              throws IOException {
     NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
-    if (isInSafeMode())
+    if (enforceSafeMode && isInSafeMode())
       throw new SafeModeException("Cannot delete " + src, safeMode);
     Block deletedBlocks[] = dir.delete(src);
     if (deletedBlocks != null) {

+ 13 - 29
src/java/org/apache/hadoop/dfs/JspHelper.java

@@ -93,38 +93,24 @@ public class JspHelper {
     Socket s = new Socket();
     s.connect(addr, FSConstants.READ_TIMEOUT);
     s.setSoTimeout(FSConstants.READ_TIMEOUT);
-    //
-    // Xmit header info to datanode
-    //
-    DataOutputStream os = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
-    os.write(FSConstants.OP_READSKIP_BLOCK);
-    new Block(blockId, blockSize).write(os);
-    os.writeLong(offsetIntoBlock);
-    os.flush();
-
-    //
-    // Get bytes in block, set streams
-    //
-    DataInputStream in = new DataInputStream(new BufferedInputStream(s.getInputStream()));
-    long curBlockSize = in.readLong();
-    long amtSkipped = in.readLong();
-    if (curBlockSize != blockSize) {
-      throw new IOException("Recorded block size is " + blockSize + ", but datanode reports size of " + curBlockSize);
-    }
-    if (amtSkipped != offsetIntoBlock) {
-      throw new IOException("Asked for offset of " + offsetIntoBlock + ", but only received offset of " + amtSkipped);
-    }
       
-    long amtToRead = chunkSizeToView;
-    if (amtToRead + offsetIntoBlock > blockSize)
-      amtToRead = blockSize - offsetIntoBlock;
+      long amtToRead = Math.min(chunkSizeToView, blockSize - offsetIntoBlock);     
+      
+      // Use the block name for file name. 
+      DFSClient.BlockReader blockReader = 
+        DFSClient.BlockReader.newBlockReader(s, addr.toString() + ":" + blockId,
+                                             blockId, offsetIntoBlock, 
+                                             amtToRead, 
+                                             conf.getInt("io.file.buffer.size",
+                                                         4096));
+        
     byte[] buf = new byte[(int)amtToRead];
     int readOffset = 0;
     int retries = 2;
-    while (true) {
+    while ( amtToRead > 0 ) {
       int numRead;
       try {
-        numRead = in.read(buf, readOffset, (int)amtToRead);
+        numRead = blockReader.readAll(buf, readOffset, (int)amtToRead);
       }
       catch (IOException e) {
         retries--;
@@ -134,11 +120,9 @@ public class JspHelper {
       }
       amtToRead -= numRead;
       readOffset += numRead;
-      if (amtToRead == 0)
-        break;
     }
+    blockReader = null;
     s.close();
-    in.close();
     out.print(new String(buf));
   }
   public void DFSNodesStatus(ArrayList<DatanodeDescriptor> live,

+ 5 - 0
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -688,6 +688,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     return namesystem.processDistributedUpgradeCommand(comm);
   }
 
+  public BlockCrcInfo blockCrcUpgradeGetBlockLocations(Block block) 
+                                                       throws IOException {
+    return namesystem.blockCrcInfo(block, true);
+  }
+
   /** 
    * Verify request.
    * 

+ 16 - 26
src/java/org/apache/hadoop/dfs/NamenodeFsck.java

@@ -310,8 +310,9 @@ public class NamenodeFsck {
     InetSocketAddress targetAddr = null;
     TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
     Socket s = null;
-    DataInputStream in = null;
-    DataOutputStream out = null;
+    DFSClient.BlockReader blockReader = null; 
+    Block block = lblock.getBlock(); 
+
     while (s == null) {
       DatanodeInfo chosenNode;
       
@@ -336,27 +337,12 @@ public class NamenodeFsck {
         s.connect(targetAddr, FSConstants.READ_TIMEOUT);
         s.setSoTimeout(FSConstants.READ_TIMEOUT);
         
-        //
-        // Xmit header info to datanode
-        //
-        out = new DataOutputStream(new BufferedOutputStream(s.getOutputStream()));
-        out.write(FSConstants.OP_READSKIP_BLOCK);
-        lblock.getBlock().write(out);
-        out.writeLong(0L);
-        out.flush();
+        blockReader = 
+          DFSClient.BlockReader.newBlockReader(s, targetAddr.toString() + ":" + 
+                                               block.getBlockId(), 
+                                               block.getBlockId(), 0, -1,
+                                               conf.getInt("io.file.buffer.size", 4096));
         
-        //
-        // Get bytes in block, set streams
-        //
-        in = new DataInputStream(new BufferedInputStream(s.getInputStream()));
-        long curBlockSize = in.readLong();
-        long amtSkipped = in.readLong();
-        if (curBlockSize != lblock.getBlock().len) {
-          throw new IOException("Recorded block size is " + lblock.getBlock().len + ", but datanode reports size of " + curBlockSize);
-        }
-        if (amtSkipped != 0L) {
-          throw new IOException("Asked for offset of " + 0L + ", but only received offset of " + amtSkipped);
-        }
       }  catch (IOException ex) {
         // Put chosen node into dead list, continue
         LOG.info("Failed to connect to " + targetAddr + ":" + ex);
@@ -370,22 +356,26 @@ public class NamenodeFsck {
         s = null;
       }
     }
-    if (in == null) {
+    if (blockReader == null) {
       throw new Exception("Could not open data stream for " + lblock.getBlock().getBlockName());
     }
     byte[] buf = new byte[1024];
     int cnt = 0;
     boolean success = true;
+    long bytesRead = 0;
     try {
-      while ((cnt = in.read(buf)) > 0) {
+      while ((cnt = blockReader.read(buf, 0, buf.length)) > 0) {
         fos.write(buf, 0, cnt);
+        bytesRead += cnt;
+      }
+      if ( bytesRead != block.getNumBytes() ) {
+        throw new IOException("Recorded block size is " + block.getNumBytes() + 
+                              ", but datanode returned " +bytesRead+" bytes");
       }
     } catch (Exception e) {
       e.printStackTrace();
       success = false;
     } finally {
-      try {in.close(); } catch (Exception e1) {}
-      try {out.close(); } catch (Exception e1) {}
       try {s.close(); } catch (Exception e1) {}
     }
     if (!success)

+ 2 - 0
src/java/org/apache/hadoop/dfs/UpgradeObjectCollection.java

@@ -33,6 +33,8 @@ class UpgradeObjectCollection {
     initialize();
     // Registered distributed upgrade objects here
     // registerUpgrade(new UpgradeObject());
+    registerUpgrade(new BlockCrcUpgradeObjectNamenode());
+    registerUpgrade(new BlockCrcUpgradeObjectDatanode());
   }
 
   static class UOSignature implements Comparable<UOSignature> {

+ 44 - 0
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -22,6 +22,7 @@ import java.io.*;
 import java.util.Enumeration;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
+import java.net.Socket;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
@@ -451,4 +452,47 @@ public class FileUtil {
     }
     return tmp;
   }
+  
+  //XXX These functions should be in IO Utils rather than FileUtil
+  // Reads len bytes in a loop.
+  public static void readFully( InputStream in, byte buf[],
+                                int off, int len ) throws IOException {
+    int toRead = len;
+    while ( toRead > 0 ) {
+      int ret = in.read( buf, off, toRead );
+      if ( ret < 0 ) {
+        throw new IOException( "Premeture EOF from inputStream");
+      }
+      toRead -= ret;
+      off += ret;
+    }
+  }
+  
+  public static void closeSocket( Socket sock ) {
+    // avoids try { close() } dance
+    if ( sock != null ) {
+      try {
+       sock.close();
+      } catch ( IOException ignored ) {
+      }
+    }
+  }
+  public static void closeStream( InputStream in ) {
+    // avoids try { close() } dance
+    if ( in != null ) {
+      try {
+        in.close();
+      } catch ( IOException ignored ) {
+      }
+    }
+  }
+  public static void closeStream( OutputStream out ) {
+    // avoids try { close() } dance
+    if ( out != null ) {
+      try {
+        out.close();
+      } catch ( IOException ignored ) {
+      }
+    }
+  }
 }

+ 1 - 0
src/java/org/apache/hadoop/fs/FsShell.java

@@ -22,6 +22,7 @@ import java.util.*;
 import java.text.SimpleDateFormat;
 
 import org.apache.hadoop.conf.*;
+import org.apache.hadoop.dfs.ChecksumDistributedFileSystem;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.util.ToolBase;
 

+ 12 - 4
src/test/org/apache/hadoop/dfs/TestDFSShell.java

@@ -42,11 +42,15 @@ public class TestDFSShell extends TestCase {
 
   public void testCopyToLocal() throws IOException {
     Configuration conf = new Configuration();
+    /* This tests some properties of ChecksumFileSystem as well.
+     * Make sure that we create ChecksumDFS */
+    conf.set("fs.hdfs.impl",
+             "org.apache.hadoop.dfs.ChecksumDistributedFileSystem");
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
-               fs instanceof DistributedFileSystem);
-    DistributedFileSystem dfs = (DistributedFileSystem)fs;
+               fs instanceof ChecksumDistributedFileSystem);
+    ChecksumDistributedFileSystem dfs = (ChecksumDistributedFileSystem)fs;
     FsShell shell = new FsShell();
     shell.setConf(conf);
 
@@ -132,11 +136,15 @@ public class TestDFSShell extends TestCase {
    */
   public void testDFSShell() throws IOException {
     Configuration conf = new Configuration();
+    /* This tests some properties of ChecksumFileSystem as well.
+     * Make sure that we create ChecksumDFS */
+    conf.set("fs.hdfs.impl",
+             "org.apache.hadoop.dfs.ChecksumDistributedFileSystem");
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
     FileSystem fs = cluster.getFileSystem();
     assertTrue("Not a HDFS: "+fs.getUri(),
-               fs instanceof DistributedFileSystem);
-    DistributedFileSystem fileSys = (DistributedFileSystem)fs;
+            fs instanceof ChecksumDistributedFileSystem);
+    ChecksumDistributedFileSystem fileSys = (ChecksumDistributedFileSystem)fs;
     FsShell shell = new FsShell();
     shell.setConf(conf);
 

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestDecommission.java

@@ -117,7 +117,7 @@ public class TestDecommission extends TestCase {
     assertTrue("Not HDFS:"+fileSys.getUri(), fileSys instanceof DistributedFileSystem);
         
     DFSClient.DFSDataInputStream dis = (DFSClient.DFSDataInputStream) 
-      ((DistributedFileSystem)fileSys).getRawFileSystem().open(name);
+      ((DistributedFileSystem)fileSys).open(name);
     Collection<LocatedBlock> dinfo = dis.getAllBlocks();
 
     for (LocatedBlock blk : dinfo) { // for each block

+ 2 - 0
src/test/org/apache/hadoop/dfs/TestFSInputChecker.java

@@ -214,6 +214,8 @@ public class TestFSInputChecker extends TestCase {
     Configuration conf = new Configuration();
     conf.setLong("dfs.block.size", BLOCK_SIZE);
     conf.setInt("io.bytes.per.checksum", BYTES_PER_SUM);
+    conf.set("fs.hdfs.impl",
+             "org.apache.hadoop.dfs.ChecksumDistributedFileSystem");
     Random rand = new Random(seed);
     rand.nextBytes(expected);
 

+ 2 - 0
src/test/org/apache/hadoop/dfs/TestFSOutputSummer.java

@@ -113,6 +113,8 @@ public class TestFSOutputSummer extends TestCase {
     Configuration conf = new Configuration();
     conf.setLong("dfs.block.size", BLOCK_SIZE);
     conf.setInt("io.bytes.per.checksum", BYTES_PER_CHECKSUM);
+    conf.set("fs.hdfs.impl",
+             "org.apache.hadoop.dfs.ChecksumDistributedFileSystem");      
     MiniDFSCluster cluster = new MiniDFSCluster(
         conf, NUM_OF_DATANODES, true, null);
     fileSys = cluster.getFileSystem();

+ 1 - 0
src/test/org/apache/hadoop/dfs/TestSmallBlock.java

@@ -81,6 +81,7 @@ public class TestSmallBlock extends TestCase {
    */
   public void testSmallBlock() throws IOException {
     Configuration conf = new Configuration();
+    conf.set("io.bytes.per.checksum", "1");
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
     FileSystem fileSys = cluster.getFileSystem();
     try {

+ 2 - 0
src/test/org/apache/hadoop/mapred/TestMiniMRDFSCaching.java

@@ -36,6 +36,8 @@ public class TestMiniMRDFSCaching extends TestCase {
     FileSystem fileSys = null;
     try {
       JobConf conf = new JobConf();
+      conf.set("fs.hdfs.impl",
+               "org.apache.hadoop.dfs.ChecksumDistributedFileSystem");      
       dfs = new MiniDFSCluster(conf, 1, true, null);
       fileSys = dfs.getFileSystem();
       mr = new MiniMRCluster(2, fileSys.getName(), 4);

Niektoré súbory nie sú zobrazené, pretože je v týchto rozdielových dátach zmenené mnoho súborov