incubator-crunch-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Shawn Smith <Shawn.Sm...@bazaarvoice.com>
Subject Crunch with Elastic MapReduce
Date Tue, 14 Aug 2012 22:55:36 GMT
Has anyone tried using Crunch with Amazon Elastic MapReduce?  I've run into a few issues, and
I thought I'd share my experiences so far:

1. A typical Elastic MapReduce job uses S3 input and output files (w/Amazon's customized Native
S3 File System) and HDFS intermediate files.  This doesn't work with Crunch calls to FileSystem.get(Configuration)
that assume the default file system (HDFS).  Example stack trace:

Exception in thread "main" java.lang.IllegalArgumentException: This file system object (hdfs://10.114.37.65:9000)
does not support access to the request path 's3://test-bucket/test/Input.avro' You possibly
called FileSystem.get(conf) when you should have called FileSystem.get(uri, conf) to obtain
a file system supporting your path.

        at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:381)
        at org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:129)
        at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:513)
        at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:767)
        at org.apache.crunch.io.SourceTargetHelper.getPathSize(SourceTargetHelper.java:44)

It looks like switching to Path.getFileSystem(Configuration) throughout allows mixing S3 and
HDFS files.

2. CrunchJob.handleMultiPaths() calls FileSystem.rename() which works only if the source and
destination use the same file system.  This breaks the final upload of the output files from
HDFS to S3.  I've created https://issues.apache.org/jira/browse/CRUNCH-47 and attached a patch
with a crude workaround.  Note that when I ran into this the final jobs in my pipeline all
failed, but without any error messages/stack traces in the logs pointing to the actual problem.
 This might have been my fault because I hadn't called Pipeline.enableDebug() on that run.

3. EMR Hadoop 1.0.3 includes Avro 1.5.3 which apparently takes precedence over Crunch's Avro
1.7.0.  I didn't mess around with trying to get my classes in the class path first…  Instead
I used the maven-shade-plugin in my job's build to shade Avro 1.7.0 from "org.apache.avro.*"
to "shaded.org.apache.avro.*" so it wouldn't conflict with the EMR version of Avro.  Example
exception (you can see the Avro source code line numbers correspond to version 1.5.3):

2012-08-13 06:50:57,547 WARN org.apache.hadoop.mapred.Child (main): Error running child
java.lang.RuntimeException: java.lang.NoSuchMethodException: org.apache.avro.mapred.Pair.<init>()
at org.apache.avro.specific.SpecificDatumReader.newInstance(SpecificDatumReader.java:101)
at org.apache.avro.specific.SpecificDatumReader.newRecord(SpecificDatumReader.java:56)

4. EMR Hadoop 1.0.3 includes two different versions of SLF4J in the class path: 1.4.3 and
1.6.4.  As a result, jobs that use SLF4J will fail non-deterministically when a particular
run uses slf4j-api-1.4.3.jar with slf4j-log4j12-1.6.4.jar, as described in the SLF4J FAQ (http://www.slf4j.org/faq.html#IllegalAccessError).
 It looks like you can workaround the problem by using shaded SLF4J jars and not relying on
the ones provided by the Hadoop distribution.  The stack trace looks something like this:

Exception in thread "main" java.lang.IllegalAccessError: tried to access field org.slf4j.impl.StaticLoggerBinder.SINGLETON
from class org.slf4j.LoggerFactory
at org.slf4j.LoggerFactory.<clinit>(LoggerFactory.java:60)
at com.bazaarvoice.etl.mapper.ClientMapper.<clinit>(ClientMapper.java:30)
at com.bazaarvoice.etl.hadoop.BulkTransformCommand.run(BulkTransformCommand.java:54)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
at com.bazaarvoice.etl.hadoop.BulkTransformCommand.main(BulkTransformCommand.java:33)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at org.apache.hadoop.util.RunJar.main(RunJar.java:187)

Thanks,
Shawn Smith

Mime
View raw message