瀏覽代碼

svn merge -c 932042 from branch-0.20 for HDFS-1041.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1.0@1332486 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 年之前
父節點
當前提交
4f4a91b7f9
共有 3 個文件被更改,包括 52 次插入15 次删除
  1. 3 0
      CHANGES.txt
  2. 18 15
      src/hdfs/org/apache/hadoop/hdfs/DFSClient.java
  3. 31 0
      src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java

+ 3 - 0
CHANGES.txt

@@ -55,6 +55,9 @@ Release 1.0.3 - unreleased
     HDFS-3316. The tar ball doesn't include jsvc any more.
     (Owen O'Malley via mattf)
 
+    HDFS-1041. DFSClient.getFileChecksum(..) should retry if connection to
+    the first datanode fails.  (szetszwo)
+
 Release 1.0.2 - 2012.03.24
 
   NEW FEATURES

+ 18 - 15
src/hdfs/org/apache/hadoop/hdfs/DFSClient.java

@@ -532,7 +532,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     return hints;
   }
 
-  private static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
+  static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
       String src, long start, long length) throws IOException {
     try {
       return namenode.getBlockLocations(src, start, length);
@@ -929,25 +929,28 @@ public class DFSClient implements FSConstants, java.io.Closeable {
      
       boolean done = false;
       for(int j = 0; !done && j < datanodes.length; j++) {
-        //connect to a datanode
-        final Socket sock = socketFactory.createSocket();
-        NetUtils.connect(sock, 
-                         NetUtils.createSocketAddr(datanodes[j].getName()),
-                         timeout);
-        sock.setSoTimeout(timeout);
+        Socket sock = null;
+        DataOutputStream out = null;
+        DataInputStream in = null;
+        
+        try {
+          //connect to a datanode
+          sock = socketFactory.createSocket();
+          NetUtils.connect(sock,
+              NetUtils.createSocketAddr(datanodes[j].getName()), timeout);
+          sock.setSoTimeout(timeout);
 
-        DataOutputStream out = new DataOutputStream(
-            new BufferedOutputStream(NetUtils.getOutputStream(sock), 
-                                     DataNode.SMALL_BUFFER_SIZE));
-        DataInputStream in = new DataInputStream(NetUtils.getInputStream(sock));
+          out = new DataOutputStream(
+              new BufferedOutputStream(NetUtils.getOutputStream(sock), 
+                                       DataNode.SMALL_BUFFER_SIZE));
+          in = new DataInputStream(NetUtils.getInputStream(sock));
 
-        // get block MD5
-        try {
           if (LOG.isDebugEnabled()) {
             LOG.debug("write to " + datanodes[j].getName() + ": "
-                + DataTransferProtocol.OP_BLOCK_CHECKSUM +
-                ", block=" + block);
+                + DataTransferProtocol.OP_BLOCK_CHECKSUM + ", block=" + block);
           }
+
+          // get block MD5
           out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
           out.write(DataTransferProtocol.OP_BLOCK_CHECKSUM);
           out.writeLong(block.getBlockId());

+ 31 - 0
src/test/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -478,4 +478,35 @@ public class TestDFSClientRetries extends TestCase {
       server.stop();
     }
   }
+
+  public void testGetFileChecksum() throws Exception {
+    final String f = "/testGetFileChecksum";
+    final Path p = new Path(f);
+
+    final Configuration conf = new Configuration();
+    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    try {
+      cluster.waitActive();
+
+      //create a file
+      final FileSystem fs = cluster.getFileSystem();
+      DFSTestUtil.createFile(fs, p, 1L << 20, (short)3, 20100402L);
+
+      //get checksum
+      final FileChecksum cs1 = fs.getFileChecksum(p);
+      assertTrue(cs1 != null);
+
+      //stop the first datanode
+      final List<LocatedBlock> locatedblocks = DFSClient.callGetBlockLocations(
+          cluster.getNameNode(), f, 0, Long.MAX_VALUE).getLocatedBlocks();
+      final DatanodeInfo first = locatedblocks.get(0).getLocations()[0];
+      cluster.stopDataNode(first.getName());
+
+      //get checksum again
+      final FileChecksum cs2 = fs.getFileChecksum(p);
+      assertEquals(cs1, cs2);
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }