Author: tucu
Date: Mon Jul 9 19:19:00 2012
New Revision: 1359345
URL: http://svn.apache.org/viewvc?rev=1359345&view=rev
Log:
MAPREDUCE-3993. Graceful handling of codec errors during decompression (kkambatl via tucu)
Modified:
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java
Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java?rev=1359345&r1=1359344&r2=1359345&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
(original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java
Mon Jul 9 19:19:00 2012
@@ -154,6 +154,28 @@ public class IOUtils {
}
/**
+ * Utility wrapper for reading from {@link InputStream}. It catches any errors
+ * thrown by the underlying stream (either IO or decompression-related), and
+ * re-throws as an IOException.
+ *
+ * @param is - InputStream to be read from
+ * @param buf - buffer the data is read into
+ * @param off - offset within buf
+ * @param len - amount of data to be read
+ * @return number of bytes read
+ */
+ public static int wrappedReadForCompressedData(InputStream is, byte[] buf,
+ int off, int len) throws IOException {
+ try {
+ return is.read(buf, off, len);
+ } catch (IOException ie) {
+ throw ie;
+ } catch (Throwable t) {
+ throw new IOException("Error while reading compressed data", t);
+ }
+ }
+
+ /**
* Reads len bytes in a loop.
*
* @param in InputStream to read from
Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java?rev=1359345&r1=1359344&r2=1359345&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java
(original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java
Mon Jul 9 19:19:00 2012
@@ -29,6 +29,7 @@ import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
+import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Test;
import org.mockito.Mockito;
@@ -152,4 +153,26 @@ public class TestIOUtils {
}
}
}
+
+ @Test
+ public void testWrappedReadForCompressedData() throws IOException {
+ byte[] buf = new byte[2];
+ InputStream mockStream = Mockito.mock(InputStream.class);
+ Mockito.when(mockStream.read(buf, 0, 1)).thenReturn(1);
+ Mockito.when(mockStream.read(buf, 0, 2)).thenThrow(
+ new java.lang.InternalError());
+
+ try {
+ assertEquals("Check expected value", 1,
+ IOUtils.wrappedReadForCompressedData(mockStream, buf, 0, 1));
+ } catch (IOException ioe) {
+ fail("Unexpected error while reading");
+ }
+ try {
+ IOUtils.wrappedReadForCompressedData(mockStream, buf, 0, 2);
+ } catch (IOException ioe) {
+ GenericTestUtils.assertExceptionContains(
+ "Error while reading compressed data", ioe);
+ }
+ }
}
|