浏览代码

HDFS-9147. Fix the setting of visibleLength in ExternalBlockReader. (Colin P. McCabe via Lei (Eddy) Xu)

(cherry picked from commit e5992ef4df63fbc6a6b8e357b32c647e7837c662)
Lei Xu 9 年之前
父节点
当前提交
213ff984ee

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -392,6 +392,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         Constructor<? extends ReplicaAccessorBuilder> ctor =
             cls.getConstructor();
         ReplicaAccessorBuilder builder = ctor.newInstance();
+        long visibleLength = startOffset + length;
         ReplicaAccessor accessor = builder.
             setAllowShortCircuitReads(allowShortCircuitLocalReads).
             setBlock(block.getBlockId(), block.getBlockPoolId()).
@@ -401,7 +402,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             setConfiguration(configuration).
             setFileName(fileName).
             setVerifyChecksum(verifyChecksum).
-            setVisibleLength(length).
+            setVisibleLength(visibleLength).
             build();
         if (accessor == null) {
           if (LOG.isTraceEnabled()) {
@@ -409,7 +410,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
                 cls.getName());
           }
         } else {
-          return new ExternalBlockReader(accessor, length, startOffset);
+          return new ExternalBlockReader(accessor, visibleLength, startOffset);
         }
       } catch (Throwable t) {
         LOG.warn("Failed to construct new object of type " +

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java

@@ -68,7 +68,8 @@ public final class ExternalBlockReader implements BlockReader {
     if (n <= 0) {
       return 0;
     }
-    // You can't skip past the end of the replica.
+    // You can't skip past the last offset that we want to read with this
+    // block reader.
     long oldPos = pos;
     pos += n;
     if (pos > visibleLength) {
@@ -79,12 +80,11 @@ public final class ExternalBlockReader implements BlockReader {
 
   @Override
   public int available() throws IOException {
-    // We return the amount of bytes that we haven't read yet from the
-    // replica, based on our current position.  Some of the other block
-    // readers return a shorter length than that.  The only advantage to
-    // returning a shorter length is that the DFSInputStream will
-    // trash your block reader and create a new one if someone tries to
-    // seek() beyond the available() region.
+    // We return the amount of bytes between the current offset and the visible
+    // length.  Some of the other block readers return a shorter length than
+    // that.  The only advantage to returning a shorter length is that the
+    // DFSInputStream will trash your block reader and create a new one if
+    // someone tries to seek() beyond the available() region.
     long diff = visibleLength - pos;
     if (diff > Integer.MAX_VALUE) {
       return Integer.MAX_VALUE;

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

@@ -1097,6 +1097,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9107. Prevent NN's unrecoverable death spiral after full GC (Daryn
     Sharp via Colin P. McCabe)
 
+    HDFS-9147. Fix the setting of visibleLength in ExternalBlockReader. (Colin
+    P. McCabe via Lei (Eddy) Xu)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestExternalBlockReader.java

@@ -190,7 +190,7 @@ public class TestExternalBlockReader {
             "than 0 at " + pos);
         return 0;
       }
-      int i = 0, nread = 0, ipos;
+      int i = off, nread = 0, ipos;
       for (ipos = (int)pos;
            (ipos < contents.length) && (nread < len);
            ipos++) {
@@ -280,7 +280,10 @@ public class TestExternalBlockReader {
       HdfsDataInputStream stream =
           (HdfsDataInputStream)dfs.open(new Path("/a"));
       byte buf[] = new byte[TEST_LENGTH];
-      IOUtils.readFully(stream, buf, 0, TEST_LENGTH);
+      stream.seek(1000);
+      IOUtils.readFully(stream, buf, 1000, TEST_LENGTH - 1000);
+      stream.seek(0);
+      IOUtils.readFully(stream, buf, 0, 1000);
       byte expected[] = DFSTestUtil.
           calculateFileContentsFromSeed(SEED, TEST_LENGTH);
       ReadStatistics stats = stream.getReadStatistics();
@@ -293,7 +296,7 @@ public class TestExternalBlockReader {
       Assert.assertNotNull(block);
       LinkedList<SyntheticReplicaAccessor> accessorList = accessors.get(uuid);
       Assert.assertNotNull(accessorList);
-      Assert.assertEquals(2, accessorList.size());
+      Assert.assertEquals(3, accessorList.size());
       SyntheticReplicaAccessor accessor = accessorList.get(0);
       Assert.assertTrue(accessor.builder.allowShortCircuit);
       Assert.assertEquals(block.getBlockPoolId(),
@@ -307,7 +310,7 @@ public class TestExternalBlockReader {
           accessor.getGenerationStamp());
       Assert.assertTrue(accessor.builder.verifyChecksum);
       Assert.assertEquals(1024L, accessor.builder.visibleLength);
-      Assert.assertEquals(1024L, accessor.totalRead);
+      Assert.assertEquals(24L, accessor.totalRead);
       Assert.assertEquals("", accessor.getError());
       Assert.assertEquals(1, accessor.numCloses);
       byte[] tempBuf = new byte[5];