You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2012/04/25 21:25:46 UTC
svn commit: r1330503 - in
/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./
CHANGES.txt src/main/java/
src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java
src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
Author: szetszwo
Date: Wed Apr 25 19:25:45 2012
New Revision: 1330503
URL: http://svn.apache.org/viewvc?rev=1330503&view=rev
Log:
svn merge -c 1330500 from trunk for HDFS-3318. Use BoundedInputStream in ByteRangeInputStream, otherwise, it hangs on transfers >2 GB.
Modified:
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/ (props changed)
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/ (props changed)
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
Propchange: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/
------------------------------------------------------------------------------
Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs:r1330500
Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1330503&r1=1330502&r2=1330503&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Apr 25 19:25:45 2012
@@ -776,6 +776,9 @@ Release 0.23.3 - UNRELEASED
HDFS-3312. In HftpFileSystem, the namenode URI is non-secure but the
delegation tokens have to use secure URI. (Daryn Sharp via szetszwo)
+ HDFS-3318. Use BoundedInputStream in ByteRangeInputStream, otherwise, it
+ hangs on transfers >2 GB. (Daryn Sharp via szetszwo)
+
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES
Propchange: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/
------------------------------------------------------------------------------
Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java:r1330500
Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java?rev=1330503&r1=1330502&r2=1330503&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java Wed Apr 25 19:25:45 2012
@@ -23,6 +23,7 @@ import java.io.InputStream;
import java.net.HttpURLConnection;
import java.net.URL;
+import org.apache.commons.io.input.BoundedInputStream;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
@@ -106,8 +107,14 @@ public abstract class ByteRangeInputStre
checkResponseCode(connection);
final String cl = connection.getHeaderField(StreamFile.CONTENT_LENGTH);
- filelength = (cl == null) ? -1 : Long.parseLong(cl);
- in = connection.getInputStream();
+ if (cl == null) {
+ throw new IOException(StreamFile.CONTENT_LENGTH+" header is missing");
+ }
+ final long streamlength = Long.parseLong(cl);
+ filelength = startPos + streamlength;
+ // Java has a bug with >2GB request streams. It won't bounds check
+ // the reads so the transfer blocks until the server times out
+ in = new BoundedInputStream(connection.getInputStream(), streamlength);
resolvedURL.setURL(getResolvedUrl(connection));
status = StreamStatus.NORMAL;
@@ -116,21 +123,27 @@ public abstract class ByteRangeInputStre
return in;
}
- private void update(final boolean isEOF, final int n)
- throws IOException {
- if (!isEOF) {
+ private int update(final int n) throws IOException {
+ if (n != -1) {
currentPos += n;
} else if (currentPos < filelength) {
throw new IOException("Got EOF but currentPos = " + currentPos
+ " < filelength = " + filelength);
}
+ return n;
}
+ @Override
public int read() throws IOException {
final int b = getInputStream().read();
- update(b == -1, 1);
+ update((b == -1) ? -1 : 1);
return b;
}
+
+ @Override
+ public int read(byte b[], int off, int len) throws IOException {
+ return update(getInputStream().read(b, off, len));
+ }
/**
* Seek to the given offset from the start of the file.
Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java?rev=1330503&r1=1330502&r2=1330503&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestByteRangeInputStream.java Wed Apr 25 19:25:45 2012
@@ -31,6 +31,7 @@ import java.io.InputStream;
import java.net.HttpURLConnection;
import java.net.URL;
+import org.apache.hadoop.hdfs.server.namenode.StreamFile;
import org.junit.Test;
public class TestByteRangeInputStream {
@@ -84,6 +85,11 @@ public static class MockHttpURLConnectio
public void setResponseCode(int resCode) {
responseCode = resCode;
}
+
+ @Override
+ public String getHeaderField(String field) {
+ return (field.equalsIgnoreCase(StreamFile.CONTENT_LENGTH)) ? "65535" : null;
+ }
}
@Test