Ver código fonte

HDFS-3492. Fix some misuses of InputStream#skip. Contributed by Colin Patrick McCabe.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1347192 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 anos atrás
pai
commit
bef65d0601

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/InputStreamEntity.java

@@ -42,10 +42,7 @@ public class InputStreamEntity implements StreamingOutput {
 
   @Override
   public void write(OutputStream os) throws IOException {
-    long skipped = is.skip(offset);
-    if (skipped < offset) {
-      throw new IOException("Requested offset beyond stream size");
-    }
+    IOUtils.skipFully(is, offset);
     if (len == -1) {
       IOUtils.copyBytes(is, os, 4096, true);
     } else {

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

@@ -310,6 +310,9 @@ Branch-2 ( Unreleased changes )
     HDFS-3505. DirectoryScanner does not join all threads in shutdown.
     (Colin Patrick McCabe via eli)
 
+    HDFS-3492. Fix some misuses of InputStream#skip (Colin Patrick McCabe
+    via todd)
+
 Release 2.0.0-alpha - 05-23-2012
 
   INCOMPATIBLE CHANGES

+ 7 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.util.DirectBufferPool;
 import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -315,23 +316,10 @@ class BlockReaderLocal implements BlockReader {
     boolean success = false;
     try {
       // Skip both input streams to beginning of the chunk containing startOffset
-      long toSkip = firstChunkOffset;
-      while (toSkip > 0) {
-        long skipped = dataIn.skip(toSkip);
-        if (skipped == 0) {
-          throw new IOException("Couldn't initialize input stream");
-        }
-        toSkip -= skipped;
-      }
+      IOUtils.skipFully(dataIn, firstChunkOffset);
       if (checksumIn != null) {
         long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
-        while (checkSumOffset > 0) {
-          long skipped = checksumIn.skip(checkSumOffset);
-          if (skipped == 0) {
-            throw new IOException("Couldn't initialize checksum input stream");
-          }
-          checkSumOffset -= skipped;
-        }
+        IOUtils.skipFully(dataIn, checkSumOffset);
       }
       success = true;
     } finally {
@@ -636,17 +624,9 @@ class BlockReaderLocal implements BlockReader {
     slowReadBuff.position(slowReadBuff.limit());
     checksumBuff.position(checksumBuff.limit());
   
-    long dataSkipped = dataIn.skip(toskip);
-    if (dataSkipped != toskip) {
-      throw new IOException("skip error in data input stream");
-    }
-    long checkSumOffset = (dataSkipped / bytesPerChecksum) * checksumSize;
-    if (checkSumOffset > 0) {
-      long skipped = checksumIn.skip(checkSumOffset);
-      if (skipped != checkSumOffset) {
-        throw new IOException("skip error in checksum input stream");
-      }
-    }
+    IOUtils.skipFully(dataIn, toskip);
+    long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
+    IOUtils.skipFully(checksumIn, checkSumOffset);
 
     // read into the middle of the chunk
     if (skipBuf == null) {
@@ -701,4 +681,4 @@ class BlockReaderLocal implements BlockReader {
   public boolean hasSentStatusCode() {
     return false;
   }
-}
+}

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
@@ -2289,9 +2290,11 @@ public abstract class FSEditLogOp {
           // 0xff, we want to skip over that region, because there's nothing
           // interesting there.
           long numSkip = e.getNumAfterTerminator();
-          if (in.skip(numSkip) < numSkip) {
+          try {
+            IOUtils.skipFully(in, numSkip);
+          } catch (IOException t) {
             FSImage.LOG.error("Failed to skip " + numSkip + " bytes of " +
-              "garbage after an OP_INVALID.  Unexpected early EOF.");
+              "garbage after an OP_INVALID.  Unexpected early EOF.", t);
             return null;
           }
         } catch (IOException e) {

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
@@ -94,8 +95,7 @@ public class TestShortCircuitLocalRead {
     // Now read using a different API.
     actual = new byte[expected.length-readOffset];
     stm = fs.open(name);
-    long skipped = stm.skip(readOffset);
-    Assert.assertEquals(skipped, readOffset);
+    IOUtils.skipFully(stm, readOffset);
     //Read a small number of bytes first.
     int nread = stm.read(actual, 0, 3);
     nread += stm.read(actual, nread, 2);
@@ -123,8 +123,7 @@ public class TestShortCircuitLocalRead {
 
     ByteBuffer actual = ByteBuffer.allocate(expected.length - readOffset);
 
-    long skipped = stm.skip(readOffset);
-    Assert.assertEquals(skipped, readOffset);
+    IOUtils.skipFully(stm, readOffset);
 
     actual.limit(3);
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
 import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -686,7 +687,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public synchronized InputStream getBlockInputStream(ExtendedBlock b,
       long seekOffset) throws IOException {
     InputStream result = getBlockInputStream(b);
-    result.skip(seekOffset);
+    IOUtils.skipFully(result, seekOffset);
     return result;
   }