HADOOP-8423. MapFile.Reader.get() crashes jvm or throws EOFException on Snappy or LZO block-compressed data. Contributed by Todd Lipcon. (harsh)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1359866 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
fb4a6ac9e9
commit
caf9fe2f13
@ -242,6 +242,10 @@ Branch-2 ( Unreleased changes )
|
||||
HADOOP-8533. Remove parallel call ununsed capability in RPC.
|
||||
(Brandon Li via suresh)
|
||||
|
||||
HADOOP-8423. MapFile.Reader.get() crashes jvm or throws
|
||||
EOFException on Snappy or LZO block-compressed data
|
||||
(todd via harsh)
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HADOOP-8372. NetUtils.normalizeHostName() incorrectly handles hostname
|
||||
|
@ -127,6 +127,8 @@ protected int getCompressedData() throws IOException {
|
||||
}
|
||||
|
||||
public void resetState() throws IOException {
|
||||
originalBlockSize = 0;
|
||||
noUncompressedBytes = 0;
|
||||
super.resetState();
|
||||
}
|
||||
|
||||
|
@ -46,6 +46,7 @@
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.MapFile;
|
||||
import org.apache.hadoop.io.RandomDatum;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
@ -68,6 +69,7 @@
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
@ -514,6 +516,50 @@ private static void sequenceFileCodecTest(Configuration conf, int lines,
|
||||
LOG.info("SUCCESS! Completed SequenceFileCodecTest with codec \"" + codecClass + "\"");
|
||||
}
|
||||
|
||||
/**
|
||||
* Regression test for HADOOP-8423: seeking in a block-compressed
|
||||
* stream would not properly reset the block decompressor state.
|
||||
*/
|
||||
@Test
|
||||
public void testSnappyMapFile() throws Exception {
|
||||
Assume.assumeTrue(SnappyCodec.isNativeCodeLoaded());
|
||||
codecTestMapFile(SnappyCodec.class, CompressionType.BLOCK, 100);
|
||||
}
|
||||
|
||||
private void codecTestMapFile(Class<? extends CompressionCodec> clazz,
|
||||
CompressionType type, int records) throws Exception {
|
||||
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
LOG.info("Creating MapFiles with " + records +
|
||||
" records using codec " + clazz.getSimpleName());
|
||||
Path path = new Path(new Path(
|
||||
System.getProperty("test.build.data", "/tmp")),
|
||||
clazz.getSimpleName() + "-" + type + "-" + records);
|
||||
|
||||
LOG.info("Writing " + path);
|
||||
createMapFile(conf, fs, path, clazz.newInstance(), type, records);
|
||||
MapFile.Reader reader = new MapFile.Reader(path, conf);
|
||||
Text key1 = new Text("002");
|
||||
assertNotNull(reader.get(key1, new Text()));
|
||||
Text key2 = new Text("004");
|
||||
assertNotNull(reader.get(key2, new Text()));
|
||||
}
|
||||
|
||||
private static void createMapFile(Configuration conf, FileSystem fs, Path path,
|
||||
CompressionCodec codec, CompressionType type, int records) throws IOException {
|
||||
MapFile.Writer writer =
|
||||
new MapFile.Writer(conf, path,
|
||||
MapFile.Writer.keyClass(Text.class),
|
||||
MapFile.Writer.valueClass(Text.class),
|
||||
MapFile.Writer.compression(type, codec));
|
||||
Text key = new Text();
|
||||
for (int j = 0; j < records; j++) {
|
||||
key.set(String.format("%03d", j));
|
||||
writer.append(key, key);
|
||||
}
|
||||
writer.close();
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
int count = 10000;
|
||||
String codecClass = "org.apache.hadoop.io.compress.DefaultCodec";
|
||||
|
Loading…
Reference in New Issue
Block a user