|
@@ -19,12 +19,14 @@
|
|
|
package org.apache.hadoop.fs.s3native;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InputStream;
|
|
|
import java.net.URI;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystemContractBaseTest;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.s3native.NativeS3FileSystem.NativeS3FsInputStream;
|
|
|
|
|
|
public abstract class NativeS3FileSystemContractBaseTest
|
|
|
extends FileSystemContractBaseTest {
|
|
@@ -148,5 +150,79 @@ public abstract class NativeS3FileSystemContractBaseTest
|
|
|
assertEquals("Double default block size", newBlockSize,
|
|
|
fs.getFileStatus(file).getBlockSize());
|
|
|
}
|
|
|
+
|
|
|
+ public void testRetryOnIoException() throws Exception {
|
|
|
+ class TestInputStream extends InputStream {
|
|
|
+ boolean shouldThrow = false;
|
|
|
+ int throwCount = 0;
|
|
|
+ int pos = 0;
|
|
|
+ byte[] bytes;
|
|
|
+
|
|
|
+ public TestInputStream() {
|
|
|
+ bytes = new byte[256];
|
|
|
+ for (int i = 0; i < 256; i++) {
|
|
|
+ bytes[i] = (byte)i;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int read() throws IOException {
|
|
|
+ shouldThrow = !shouldThrow;
|
|
|
+ if (shouldThrow) {
|
|
|
+ throwCount++;
|
|
|
+ throw new IOException();
|
|
|
+ }
|
|
|
+ return pos++;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int read(byte[] b, int off, int len) throws IOException {
|
|
|
+ shouldThrow = !shouldThrow;
|
|
|
+ if (shouldThrow) {
|
|
|
+ throwCount++;
|
|
|
+ throw new IOException();
|
|
|
+ }
|
|
|
+
|
|
|
+ int sizeToRead = Math.min(len, 256 - pos);
|
|
|
+ for (int i = 0; i < sizeToRead; i++) {
|
|
|
+ b[i] = bytes[pos + i];
|
|
|
+ }
|
|
|
+ pos += sizeToRead;
|
|
|
+ return sizeToRead;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ final InputStream is = new TestInputStream();
|
|
|
+
|
|
|
+ class MockNativeFileSystemStore extends Jets3tNativeFileSystemStore {
|
|
|
+ @Override
|
|
|
+ public InputStream retrieve(String key, long byteRangeStart) throws IOException {
|
|
|
+ return is;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ NativeS3FsInputStream stream = new NativeS3FsInputStream(new MockNativeFileSystemStore(), null, is, "");
|
|
|
+
|
|
|
+ // Test reading methods.
|
|
|
+ byte[] result = new byte[256];
|
|
|
+ for (int i = 0; i < 128; i++) {
|
|
|
+ result[i] = (byte)stream.read();
|
|
|
+ }
|
|
|
+ for (int i = 128; i < 256; i += 8) {
|
|
|
+ byte[] temp = new byte[8];
|
|
|
+ int read = stream.read(temp, 0, 8);
|
|
|
+ assertEquals(8, read);
|
|
|
+ System.arraycopy(temp, 0, result, i, 8);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Assert correct
|
|
|
+ for (int i = 0; i < 256; i++) {
|
|
|
+ assertEquals((byte)i, result[i]);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Test to make sure the throw path was exercised.
|
|
|
+ // 144 = 128 + (128 / 8)
|
|
|
+ assertEquals(144, ((TestInputStream)is).throwCount);
|
|
|
+ }
|
|
|
|
|
|
}
|