Re: Problems with block compression using native codecs (Snappy, LZO) and MapFile.Reader.get()
JIRA entry created: https://issues.apache.org/jira/browse/HADOOP-8423 On 5/21/12, Jason B urg...@gmail.com wrote: Sorry about using attachment. The code is below for the reference. (I will also file a jira as you suggesting) package codectest; import com.hadoop.compression.lzo.LzoCodec; import java.io.IOException; import java.util.Formatter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; public class MapFileCodecTest implements Tool { private Configuration conf = new Configuration(); private void createMapFile(Configuration conf, FileSystem fs, String path, CompressionCodec codec, CompressionType type, int records) throws IOException { MapFile.Writer writer = new MapFile.Writer(conf, fs, path, Text.class, Text.class, type, codec, null); Text key = new Text(); for (int j = 0; j records; j++) { StringBuilder sb = new StringBuilder(); Formatter formatter = new Formatter(sb); formatter.format(%03d, j); key.set(sb.toString()); writer.append(key, key); } writer.close(); } private void testCodec(Configuration conf, Class? extends CompressionCodec clazz, CompressionType type, int records) throws IOException { FileSystem fs = FileSystem.getLocal(conf); try { System.out.println(Creating MapFiles with + records + records using codec + clazz.getSimpleName()); String path = clazz.getSimpleName() + records; createMapFile(conf, fs, path, clazz.newInstance(), type, records); MapFile.Reader reader = new MapFile.Reader(fs, path, conf); Text key1 = new Text(002); if (reader.get(key1, new Text()) != null) { System.out.println(1st key found); } Text key2 = new Text(004); if (reader.get(key2, new Text()) != null) { System.out.println(2nd key found); } } catch (Throwable ex) { ex.printStackTrace(); } } @Override public int run(String[] strings) throws Exception { System.out.println(Using native library + System.getProperty(java.library.path)); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 100); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 100); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 100); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 10); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 10); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 10); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 100); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 100); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 100); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 10); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 10); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 10); return 0; } @Override public void setConf(Configuration c) { this.conf = c; } @Override public Configuration getConf() { return conf; } public static void main(String[] args) throws Exception { ToolRunner.run(new MapFileCodecTest(), args); } } On 5/21/12, Todd Lipcon t...@cloudera.com wrote: Hi Jason, Sounds like a bug. Unfortunately the mailing list strips attachments. Can you file a jira in the HADOOP project, and attach your test case there? Thanks Todd On Mon, May 21, 2012 at 3:57 PM, Jason B urg...@gmail.com wrote: I am using Cloudera distribution cdh3u1. When trying to check native codecs for better decompression performance such as Snappy or LZO, I ran into issues with random access using MapFile.Reader.get(key, value) method. First call of MapFile.Reader.get() works but a second call fails. Also I am getting different exceptions depending on number of entries in a map file. With LzoCodec and 10 record file, jvm gets aborted. At the same time the DefaultCodec works fine for all cases, as well as record compression for the native codecs. I created a simple test program (attached) that creates map files locally with sizes of 10 and 100 records for three codecs: Default, Snappy, and LZO. (The test requires corresponding native library available) The summary
Re: Problems with block compression using native codecs (Snappy, LZO) and MapFile.Reader.get()
if You are getting a SIGSEG it never hurts to try a more recent JVM. 21 has many bug fixes at this point. On Tue, May 22, 2012 at 11:45 AM, Jason B urg...@gmail.com wrote: JIRA entry created: https://issues.apache.org/jira/browse/HADOOP-8423 On 5/21/12, Jason B urg...@gmail.com wrote: Sorry about using attachment. The code is below for the reference. (I will also file a jira as you suggesting) package codectest; import com.hadoop.compression.lzo.LzoCodec; import java.io.IOException; import java.util.Formatter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; public class MapFileCodecTest implements Tool { private Configuration conf = new Configuration(); private void createMapFile(Configuration conf, FileSystem fs, String path, CompressionCodec codec, CompressionType type, int records) throws IOException { MapFile.Writer writer = new MapFile.Writer(conf, fs, path, Text.class, Text.class, type, codec, null); Text key = new Text(); for (int j = 0; j records; j++) { StringBuilder sb = new StringBuilder(); Formatter formatter = new Formatter(sb); formatter.format(%03d, j); key.set(sb.toString()); writer.append(key, key); } writer.close(); } private void testCodec(Configuration conf, Class? extends CompressionCodec clazz, CompressionType type, int records) throws IOException { FileSystem fs = FileSystem.getLocal(conf); try { System.out.println(Creating MapFiles with + records + records using codec + clazz.getSimpleName()); String path = clazz.getSimpleName() + records; createMapFile(conf, fs, path, clazz.newInstance(), type, records); MapFile.Reader reader = new MapFile.Reader(fs, path, conf); Text key1 = new Text(002); if (reader.get(key1, new Text()) != null) { System.out.println(1st key found); } Text key2 = new Text(004); if (reader.get(key2, new Text()) != null) { System.out.println(2nd key found); } } catch (Throwable ex) { ex.printStackTrace(); } } @Override public int run(String[] strings) throws Exception { System.out.println(Using native library + System.getProperty(java.library.path)); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 100); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 100); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 100); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 10); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 10); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 10); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 100); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 100); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 100); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 10); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 10); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 10); return 0; } @Override public void setConf(Configuration c) { this.conf = c; } @Override public Configuration getConf() { return conf; } public static void main(String[] args) throws Exception { ToolRunner.run(new MapFileCodecTest(), args); } } On 5/21/12, Todd Lipcon t...@cloudera.com wrote: Hi Jason, Sounds like a bug. Unfortunately the mailing list strips attachments. Can you file a jira in the HADOOP project, and attach your test case there? Thanks Todd On Mon, May 21, 2012 at 3:57 PM, Jason B urg...@gmail.com wrote: I am using Cloudera distribution cdh3u1. When trying to check native codecs for better decompression performance such as Snappy or LZO, I ran into issues with random access using MapFile.Reader.get(key, value) method. First call of MapFile.Reader.get() works but a second call fails. Also I am getting different exceptions depending on number of entries in a map file. With LzoCodec and 10 record file, jvm gets aborted. At the same time the DefaultCodec works fine for all cases, as well as record compression for the native codecs. I created a simple test program (attached) that
Re: Problems with block compression using native codecs (Snappy, LZO) and MapFile.Reader.get()
This is from our production environment. Unfortunately, I cannot test this on any newer version until it is upgraded to cdh4 (0.23) But since this is chd3u1 release, it presumably already contains a lot of bug fixes back ported from 0.21. The SIGSER is just one of the issues. EOFException is raised gracefully for other cases. On 5/22/12, Edward Capriolo edlinuxg...@gmail.com wrote: if You are getting a SIGSEG it never hurts to try a more recent JVM. 21 has many bug fixes at this point. On Tue, May 22, 2012 at 11:45 AM, Jason B urg...@gmail.com wrote: JIRA entry created: https://issues.apache.org/jira/browse/HADOOP-8423 On 5/21/12, Jason B urg...@gmail.com wrote: Sorry about using attachment. The code is below for the reference. (I will also file a jira as you suggesting) package codectest; import com.hadoop.compression.lzo.LzoCodec; import java.io.IOException; import java.util.Formatter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; public class MapFileCodecTest implements Tool { private Configuration conf = new Configuration(); private void createMapFile(Configuration conf, FileSystem fs, String path, CompressionCodec codec, CompressionType type, int records) throws IOException { MapFile.Writer writer = new MapFile.Writer(conf, fs, path, Text.class, Text.class, type, codec, null); Text key = new Text(); for (int j = 0; j records; j++) { StringBuilder sb = new StringBuilder(); Formatter formatter = new Formatter(sb); formatter.format(%03d, j); key.set(sb.toString()); writer.append(key, key); } writer.close(); } private void testCodec(Configuration conf, Class? extends CompressionCodec clazz, CompressionType type, int records) throws IOException { FileSystem fs = FileSystem.getLocal(conf); try { System.out.println(Creating MapFiles with + records + records using codec + clazz.getSimpleName()); String path = clazz.getSimpleName() + records; createMapFile(conf, fs, path, clazz.newInstance(), type, records); MapFile.Reader reader = new MapFile.Reader(fs, path, conf); Text key1 = new Text(002); if (reader.get(key1, new Text()) != null) { System.out.println(1st key found); } Text key2 = new Text(004); if (reader.get(key2, new Text()) != null) { System.out.println(2nd key found); } } catch (Throwable ex) { ex.printStackTrace(); } } @Override public int run(String[] strings) throws Exception { System.out.println(Using native library + System.getProperty(java.library.path)); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 100); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 100); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 100); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 10); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 10); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 10); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 100); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 100); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 100); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 10); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 10); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 10); return 0; } @Override public void setConf(Configuration c) { this.conf = c; } @Override public Configuration getConf() { return conf; } public static void main(String[] args) throws Exception { ToolRunner.run(new MapFileCodecTest(), args); } } On 5/21/12, Todd Lipcon t...@cloudera.com wrote: Hi Jason, Sounds like a bug. Unfortunately the mailing list strips attachments. Can you file a jira in the HADOOP project, and attach your test case there? Thanks Todd On Mon, May 21, 2012 at 3:57 PM, Jason B urg...@gmail.com wrote: I am using Cloudera distribution cdh3u1. When trying to check native codecs for better decompression performance such as Snappy or LZO, I ran into issues with random access using MapFile.Reader.get(key,
Re: Problems with block compression using native codecs (Snappy, LZO) and MapFile.Reader.get()
Sorry about using attachment. The code is below for the reference. (I will also file a jira as you suggesting) package codectest; import com.hadoop.compression.lzo.LzoCodec; import java.io.IOException; import java.util.Formatter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.io.compress.SnappyCodec; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; public class MapFileCodecTest implements Tool { private Configuration conf = new Configuration(); private void createMapFile(Configuration conf, FileSystem fs, String path, CompressionCodec codec, CompressionType type, int records) throws IOException { MapFile.Writer writer = new MapFile.Writer(conf, fs, path, Text.class, Text.class, type, codec, null); Text key = new Text(); for (int j = 0; j records; j++) { StringBuilder sb = new StringBuilder(); Formatter formatter = new Formatter(sb); formatter.format(%03d, j); key.set(sb.toString()); writer.append(key, key); } writer.close(); } private void testCodec(Configuration conf, Class? extends CompressionCodec clazz, CompressionType type, int records) throws IOException { FileSystem fs = FileSystem.getLocal(conf); try { System.out.println(Creating MapFiles with + records + records using codec + clazz.getSimpleName()); String path = clazz.getSimpleName() + records; createMapFile(conf, fs, path, clazz.newInstance(), type, records); MapFile.Reader reader = new MapFile.Reader(fs, path, conf); Text key1 = new Text(002); if (reader.get(key1, new Text()) != null) { System.out.println(1st key found); } Text key2 = new Text(004); if (reader.get(key2, new Text()) != null) { System.out.println(2nd key found); } } catch (Throwable ex) { ex.printStackTrace(); } } @Override public int run(String[] strings) throws Exception { System.out.println(Using native library + System.getProperty(java.library.path)); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 100); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 100); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 100); testCodec(conf, DefaultCodec.class, CompressionType.RECORD, 10); testCodec(conf, SnappyCodec.class, CompressionType.RECORD, 10); testCodec(conf, LzoCodec.class, CompressionType.RECORD, 10); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 100); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 100); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 100); testCodec(conf, DefaultCodec.class, CompressionType.BLOCK, 10); testCodec(conf, SnappyCodec.class, CompressionType.BLOCK, 10); testCodec(conf, LzoCodec.class, CompressionType.BLOCK, 10); return 0; } @Override public void setConf(Configuration c) { this.conf = c; } @Override public Configuration getConf() { return conf; } public static void main(String[] args) throws Exception { ToolRunner.run(new MapFileCodecTest(), args); } } On 5/21/12, Todd Lipcon t...@cloudera.com wrote: Hi Jason, Sounds like a bug. Unfortunately the mailing list strips attachments. Can you file a jira in the HADOOP project, and attach your test case there? Thanks Todd On Mon, May 21, 2012 at 3:57 PM, Jason B urg...@gmail.com wrote: I am using Cloudera distribution cdh3u1. When trying to check native codecs for better decompression performance such as Snappy or LZO, I ran into issues with random access using MapFile.Reader.get(key, value) method. First call of MapFile.Reader.get() works but a second call fails. Also I am getting different exceptions depending on number of entries in a map file. With LzoCodec and 10 record file, jvm gets aborted. At the same time the DefaultCodec works fine for all cases, as well as record compression for the native codecs. I created a simple test program (attached) that creates map files locally with sizes of 10 and 100 records for three codecs: Default, Snappy, and LZO. (The test requires corresponding native library available) The summary of problems are given below: Map Size: 100 Compression: RECORD == DefaultCodec: OK SnappyCodec: OK LzoCodec: OK Map Size: 10 Compression: RECORD == DefaultCodec: