HADOOP-14072. AliyunOSS: Failed to read from stream when seek beyond the download size. Contributed by Genmao Yu
This commit is contained in:
parent
8acb376c9c
commit
cd3e59a3dc
@ -224,8 +224,10 @@ public class AliyunOSSInputStream extends FSInputStream {
|
|||||||
if (position == pos) {
|
if (position == pos) {
|
||||||
return;
|
return;
|
||||||
} else if (pos > position && pos < position + partRemaining) {
|
} else if (pos > position && pos < position + partRemaining) {
|
||||||
AliyunOSSUtils.skipFully(wrappedStream, pos - position);
|
long len = pos - position;
|
||||||
|
AliyunOSSUtils.skipFully(wrappedStream, len);
|
||||||
position = pos;
|
position = pos;
|
||||||
|
partRemaining -= len;
|
||||||
} else {
|
} else {
|
||||||
reopen(pos);
|
reopen(pos);
|
||||||
}
|
}
|
||||||
|
@ -19,8 +19,15 @@
|
|||||||
package org.apache.hadoop.fs.aliyun.oss.contract;
|
package org.apache.hadoop.fs.aliyun.oss.contract;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
|
import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
|
||||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
||||||
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Aliyun OSS contract seeking tests.
|
* Aliyun OSS contract seeking tests.
|
||||||
@ -31,4 +38,23 @@ public class TestAliyunOSSContractSeek extends AbstractContractSeekTest {
|
|||||||
protected AbstractFSContract createContract(Configuration conf) {
|
protected AbstractFSContract createContract(Configuration conf) {
|
||||||
return new AliyunOSSContract(conf);
|
return new AliyunOSSContract(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSeekBeyondDownloadSize() throws Throwable {
|
||||||
|
describe("seek and read beyond download size.");
|
||||||
|
|
||||||
|
Path byteFile = path("byte_file.txt");
|
||||||
|
// 'fs.oss.multipart.download.size' = 100 * 1024
|
||||||
|
byte[] block = dataset(100 * 1024 + 10, 0, 255);
|
||||||
|
FileSystem fs = getFileSystem();
|
||||||
|
createFile(fs, byteFile, true, block);
|
||||||
|
|
||||||
|
FSDataInputStream instream = getFileSystem().open(byteFile);
|
||||||
|
instream.seek(100 * 1024 - 1);
|
||||||
|
assertEquals(100 * 1024 - 1, instream.getPos());
|
||||||
|
assertEquals(144, instream.read());
|
||||||
|
instream.seek(100 * 1024 + 1);
|
||||||
|
assertEquals(100 * 1024 + 1, instream.getPos());
|
||||||
|
assertEquals(146, instream.read());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user