瀏覽代碼

HDFS-5762. BlockReaderLocal does not return -1 on EOF when doing zero-length reads (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1558526 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 年之前
父節點
當前提交
037a89abc5

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

@@ -674,6 +674,9 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5220. Expose group resolution time as metric (jxiang via cmccabe)
 
+    HDFS-5762. BlockReaderLocal doesn't return -1 on EOF when doing zero-length
+    reads (Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -39,6 +39,8 @@ public interface BlockReader extends ByteBufferReadable {
    * "Read should not modify user buffer before successful read"
    * because it first reads the data to user buffer and then checks
    * the checksum.
+   * Note: this must return -1 on EOF, even in the case of a 0-byte read.
+   * See HDFS-5762 for details.
    */
   int read(byte[] buf, int off, int len) throws IOException;
 

+ 25 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -328,10 +328,12 @@ class BlockReaderLocal implements BlockReader {
 
   private synchronized int drainDataBuf(ByteBuffer buf)
       throws IOException {
-    if (dataBuf == null) return 0;
+    if (dataBuf == null) return -1;
     int oldLimit = dataBuf.limit();
     int nRead = Math.min(dataBuf.remaining(), buf.remaining());
-    if (nRead == 0) return 0;
+    if (nRead == 0) {
+      return (dataBuf.remaining() == 0) ? -1 : 0;
+    }
     try {
       dataBuf.limit(dataBuf.position() + nRead);
       buf.put(dataBuf);
@@ -444,13 +446,11 @@ class BlockReaderLocal implements BlockReader {
     int total = 0;
     while (buf.hasRemaining()) {
       int nRead = dataIn.read(buf, dataPos);
-      if (nRead < 0) {
-        break;
-      }
+      if (nRead <= 0) break;
       dataPos += nRead;
       total += nRead;
     }
-    return (total == 0) ? -1 : total;
+    return (total == 0 && (dataPos == dataIn.size())) ? -1 : total;
   }
 
   /**
@@ -512,15 +512,15 @@ class BlockReaderLocal implements BlockReader {
   private synchronized int readWithBounceBuffer(ByteBuffer buf,
         boolean canSkipChecksum) throws IOException {
     int total = 0;
-    boolean eof = false;
-    while (true) {
-      int bb = drainDataBuf(buf); // drain bounce buffer if possible
+    int bb = drainDataBuf(buf); // drain bounce buffer if possible
+    if (bb >= 0) {
       total += bb;
-      int needed = buf.remaining();
-      if (eof || (needed == 0)) {
-        break;
-      } else if (buf.isDirect() && (needed >= maxReadaheadLength)
-          && ((dataPos % bytesPerChecksum) == 0)) {
+      if (buf.remaining() == 0) return total;
+    }
+    boolean eof = false;
+    do {
+      if (buf.isDirect() && (buf.remaining() >= maxReadaheadLength)
+            && ((dataPos % bytesPerChecksum) == 0)) {
         // Fast lane: try to read directly into user-supplied buffer, bypassing
         // bounce buffer.
         int oldLimit = buf.limit();
@@ -540,9 +540,13 @@ class BlockReaderLocal implements BlockReader {
         if (fillDataBuf(canSkipChecksum)) {
           eof = true;
         }
+        bb = drainDataBuf(buf); // drain bounce buffer if possible
+        if (bb >= 0) {
+          total += bb;
+        }
       }
-    }
-    return total == 0 ? -1 : total;
+    } while ((!eof) && (buf.remaining() > 0));
+    return (eof && total == 0) ? -1 : total;
   }
 
   @Override
@@ -587,8 +591,10 @@ class BlockReaderLocal implements BlockReader {
     int nRead = dataIn.read(ByteBuffer.wrap(arr, off, len), dataPos);
     if (nRead > 0) {
       dataPos += nRead;
+    } else if ((nRead == 0) && (dataPos == dataIn.size())) {
+      return -1;
     }
-    return nRead == 0 ? -1 : nRead;
+    return nRead;
   }
 
   private synchronized int readWithBounceBuffer(byte arr[], int off, int len,
@@ -599,9 +605,10 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.limit(maxReadaheadLength);
       fillDataBuf(canSkipChecksum);
     }
+    if (dataBuf.remaining() == 0) return -1;
     int toRead = Math.min(dataBuf.remaining(), len);
     dataBuf.get(arr, off, toRead);
-    return toRead == 0 ? -1 : toRead;
+    return toRead;
   }
 
   @Override

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java

@@ -604,6 +604,24 @@ public class TestBlockReaderLocal {
     }
   }
 
+  private static class TestBlockReaderLocalReadZeroBytes
+      extends BlockReaderLocalTest {
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[])
+        throws IOException {
+      byte emptyArr[] = new byte[0];
+      Assert.assertEquals(0, reader.read(emptyArr, 0, 0));
+      ByteBuffer emptyBuf = ByteBuffer.wrap(emptyArr);
+      Assert.assertEquals(0, reader.read(emptyBuf));
+      reader.skip(1);
+      Assert.assertEquals(0, reader.read(emptyArr, 0, 0));
+      Assert.assertEquals(0, reader.read(emptyBuf));
+      reader.skip(BlockReaderLocalTest.TEST_LENGTH - 1);
+      Assert.assertEquals(-1, reader.read(emptyArr, 0, 0));
+      Assert.assertEquals(-1, reader.read(emptyBuf));
+    }
+  }
+
   @Test
   public void testBlockReaderLocalOnFileWithoutChecksum()
       throws IOException {
@@ -631,6 +649,35 @@ public class TestBlockReaderLocal {
     runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
         false, 0);
   }
+  
+  @Test
+  public void testBlockReaderLocalReadZeroBytes()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadZeroBytesNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadZeroBytesNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadZeroBytesNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        false, 0);
+  }
+  
 
   @Test(timeout=60000)
   public void TestStatisticsForShortCircuitLocalRead() throws Exception {