Repository: tez Updated Branches: refs/heads/master 6cbfd1e76 -> 573ac29bd
TEZ-3212. IFile throws NegativeArraySizeException for value sizes between 1GB and 2GB (Muhammad Samir Khan via jeagles) Project: http://git-wip-us.apache.org/repos/asf/tez/repo Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/573ac29b Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/573ac29b Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/573ac29b Branch: refs/heads/master Commit: 573ac29bda098d7f2810fc53d877e5ff1b4fb801 Parents: 6cbfd1e Author: Jonathan Eagles <[email protected]> Authored: Mon Jul 31 10:34:50 2017 -0500 Committer: Jonathan Eagles <[email protected]> Committed: Mon Jul 31 10:34:50 2017 -0500 ---------------------------------------------------------------------- tez-runtime-library/findbugs-exclude.xml | 6 + .../runtime/library/common/sort/impl/IFile.java | 36 +++++- .../library/common/sort/impl/TestIFile.java | 109 +++++++++++++++++++ 3 files changed, 146 insertions(+), 5 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/tez/blob/573ac29b/tez-runtime-library/findbugs-exclude.xml ---------------------------------------------------------------------- diff --git a/tez-runtime-library/findbugs-exclude.xml b/tez-runtime-library/findbugs-exclude.xml index bcdbcc5..24fe93e 100644 --- a/tez-runtime-library/findbugs-exclude.xml +++ b/tez-runtime-library/findbugs-exclude.xml @@ -201,4 +201,10 @@ <Method name="maybeForceBuilderInitialization"/> <Bug pattern="UCF_USELESS_CONTROL_FLOW"/> </Match> + + <Match> + <Class name="org.apache.tez.runtime.library.common.sort.impl.IFile$Reader"/> + <Field name="MAX_BUFFER_SIZE"/> + <Bug pattern="MS_SHOULD_BE_FINAL"/> + </Match> </FindBugsFilter> http://git-wip-us.apache.org/repos/asf/tez/blob/573ac29b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java ---------------------------------------------------------------------- diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java index 8e3661b..b502fc9 100644 --- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java +++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java @@ -69,6 +69,7 @@ public class IFile { (byte) 'F' , (byte) 0}; private static final String INCOMPLETE_READ = "Requested to read %d got %d"; + private static final String REQ_BUFFER_SIZE_TOO_LARGE = "Size of data %d is greater than the max allowed of %d"; /** * <code>IFile.Writer</code> to write out intermediate map-outputs. @@ -480,6 +481,13 @@ public class IFile { public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY} private static final int DEFAULT_BUFFER_SIZE = 128*1024; + @VisibleForTesting + // Not final for testing + protected static int MAX_BUFFER_SIZE + = Integer.MAX_VALUE - 8; // The maximum array size is a little less than the + // max integer value. Trying to create a larger array + // will result in an OOM exception. The exact value + // is JVM dependent so setting it to max int - 8 to be safe. // Count records read from disk private long numRecordsRead = 0; @@ -782,6 +790,21 @@ public class IFile { return readRawKey(key) != KeyState.NO_KEY; } + private static byte[] createLargerArray(int currentLength) { + if (currentLength > MAX_BUFFER_SIZE) { + throw new IllegalArgumentException( + String.format(REQ_BUFFER_SIZE_TOO_LARGE, currentLength, MAX_BUFFER_SIZE)); + } + int newLength; + if (currentLength > (MAX_BUFFER_SIZE - currentLength)) { + // possible overflow: if (2*currentLength > MAX_BUFFER_SIZE) + newLength = currentLength; + } else { + newLength = currentLength << 1; + } + return new byte[newLength]; + } + public KeyState readRawKey(DataInputBuffer key) throws IOException { if (!positionToNextRecord(dataIn)) { if (LOG.isDebugEnabled()) { @@ -798,7 +821,7 @@ public class IFile { return KeyState.SAME_KEY; } if (keyBytes.length < currentKeyLength) { - keyBytes = new byte[currentKeyLength << 1]; + keyBytes = createLargerArray(currentKeyLength); } int i = readData(keyBytes, 0, currentKeyLength); if (i != currentKeyLength) { @@ -810,10 +833,13 @@ public class IFile { } public void nextRawValue(DataInputBuffer value) throws IOException { - final byte[] valBytes = - ((value.getData().length < currentValueLength) || (value.getData() == keyBytes)) - ? new byte[currentValueLength << 1] - : value.getData(); + final byte[] valBytes; + if ((value.getData().length < currentValueLength) || (value.getData() == keyBytes)) { + valBytes = createLargerArray(currentValueLength); + } else { + valBytes = value.getData(); + } + int i = readData(valBytes, 0, currentValueLength); if (i != currentValueLength) { throw new IOException(String.format(INCOMPLETE_READ, currentValueLength, i)); http://git-wip-us.apache.org/repos/asf/tez/blob/573ac29b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java ---------------------------------------------------------------------- diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java index 25e916e..f06fda3 100644 --- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java +++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java @@ -29,6 +29,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Random; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -171,6 +172,114 @@ public class TestIFile { @Test(timeout = 5000) + //test overflow + public void testExceedMaxSize() throws IOException { + final int oldMaxBufferSize = IFile.Reader.MAX_BUFFER_SIZE; + + Text shortString = new Text("string"); + Text longString = new Text("A string of length 22."); + assertEquals(22, longString.getLength()); + + Text readKey = new Text(); + Text readValue = new Text(); + DataInputBuffer keyIn = new DataInputBuffer(); + DataInputBuffer valIn = new DataInputBuffer(); + + IFile.Writer writer; + IFile.Reader reader; + FSDataOutputStream out; + + // Check Key length exceeding MAX_BUFFER_SIZE + out = localFs.create(outputPath); + writer = new IFile.Writer(defaultConf, out, + Text.class, Text.class, null, null, null, false); + writer.append(longString, shortString); + writer.close(); + + out.close(); + + // Set this to a smaller value for testing + IFile.Reader.MAX_BUFFER_SIZE = 16; + + reader = new IFile.Reader(localFs, outputPath, + null, null, null, false, 0, -1); + + try { + reader.nextRawKey(keyIn); + Assert.fail("Expected IllegalArgumentException to be thrown"); + } catch (IllegalArgumentException e) { + // test passed + } + reader.close(); + + // Check Value length exceeding MAX_BUFFER_SIZE + out = localFs.create(outputPath); + writer = new IFile.Writer(defaultConf, out, + Text.class, Text.class, null, null, null, false); + writer.append(shortString, longString); + writer.close(); + + out.close(); + + // Set this to a smaller value for testing + IFile.Reader.MAX_BUFFER_SIZE = 16; + + reader = new IFile.Reader(localFs, outputPath, + null, null, null, false, 0, -1); + + try { + reader.nextRawKey(keyIn); + reader.nextRawValue(valIn); + Assert.fail("Expected IllegalArgumentException to be thrown"); + } catch (IllegalArgumentException e) { + // test passed + } + reader.close(); + + // Check Key length not getting doubled + out = localFs.create(outputPath); + writer = new IFile.Writer(defaultConf, out, + Text.class, Text.class, null, null, null, false); + writer.append(longString, shortString); + writer.close(); + + out.close(); + + // Set this to a smaller value for testing + IFile.Reader.MAX_BUFFER_SIZE = 32; + + reader = new IFile.Reader(localFs, outputPath, + null, null, null, false, 0, -1); + + reader.nextRawKey(keyIn); + assertEquals(longString.getLength() + 1, keyIn.getData().length); + reader.close(); + + // Check Value length not getting doubled + out = localFs.create(outputPath); + writer = new IFile.Writer(defaultConf, out, + Text.class, Text.class, null, null, null, false); + writer.append(shortString, longString); + writer.close(); + + out.close(); + + // Set this to a smaller value for testing + IFile.Reader.MAX_BUFFER_SIZE = 32; + + reader = new IFile.Reader(localFs, outputPath, + null, null, null, false, 0, -1); + + reader.nextRawKey(keyIn); + reader.nextRawValue(valIn); + assertEquals(longString.getLength() + 1, valIn.getData().length); + reader.close(); + + // revert back to original value + IFile.Reader.MAX_BUFFER_SIZE = oldMaxBufferSize; + } + + @Test(timeout = 5000) //test with sorted data and repeat keys public void testWithRLEMarker() throws IOException { //Test with append(Object, Object)
