crunch-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Gabriel Reid (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CRUNCH-360) GenericData.Record avro records without schema namespace gets implicit namespace"crunch"
Date Sun, 02 Mar 2014 23:21:22 GMT

    [ https://issues.apache.org/jira/browse/CRUNCH-360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13917600#comment-13917600
] 

Gabriel Reid commented on CRUNCH-360:
-------------------------------------

Thanks for trying that out, and sorry it didn't work out. I put some more time into debugging
this issue this evening. 

About the HADOOP_USER_CLASSPATH_FIRST and mapreduce.job.user.classpath.first config properties:
I totally overlooked the fact that those won't work in the case of running with the local
job tracker (which is what you're running), as everything runs in a single JVM with the initial
classpath when the tool is started.

I tried running your example with a clean download of hadoop 2.2.0 with avro-1.7.4.jar under
share/hadoop/common/lib replaced with avro-1.7.6.jar, and I no longer got the exact issue
that you were running into. However, I got another issue, with appears to be another bug with
Avro.

I've logged the Avro bug and filed a patch under AVRO-1473. I also tried your test code again
with the patched Avro snapshot placed under share/hadoop/common/lib, and both the namespaced
and non-namespaced versions work.

To get around this issue without using namespaces in Avro means that you'll need to use a
patched/newer version of Avro for now. The HADOOP_USER_CLASSPATH_FIRST env variable and mapreduce.job.user.classpath.first
config property should help you out if you're running on an actual cluster, but running in
the local job tracker will require replacing the Avro jar manually in your hadoop installation.

> GenericData.Record avro records without schema namespace gets implicit namespace"crunch"
> ----------------------------------------------------------------------------------------
>
>                 Key: CRUNCH-360
>                 URL: https://issues.apache.org/jira/browse/CRUNCH-360
>             Project: Crunch
>          Issue Type: Bug
>            Reporter: Magnus Runesson
>         Attachments: ImplicitNamespaceSchemaIT.java, crunch-example.tar.gz
>
>
> When having avroschema without namespace crunch implicit adds the namespace "crunch"
when working with the records. Unfortunately this is not happening to the schema when reading
an avrofile with At.avroFile(Path path, Configuration conf). The schema still has no namespace.
> In my job it ends up that my job fails looking up in the schema with the following error:
> The job uses Avro GenericData.Record and gets the schema from the avro file that is read.

> 2014-02-27 09:58:14,236 WARN [main] org.apache.hadoop.mapred.YarnChild: Exception running
child : org.apache.avro.UnresolvedUnionException: Not in union [{"type":"record","name":"UsernameToUserId","namespace":"crunch","fields":[{"name":"username","type":["string","null"]},{"name":"user_id","type":["string","null"]}]},"null"]:
{"username": "XXXXXXXXXX", "user_id": "XXXXXXXXXXXXXXXXXX"}
> 	at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:561)
> 	at org.apache.avro.generic.GenericDatumWriter.resolveUnion(GenericDatumWriter.java:144)
> 	at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:71)
> 	at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:106)
> 	at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:66)
> 	at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:58)
> 	at org.apache.crunch.types.avro.SafeAvroSerialization$AvroWrapperSerializer.serialize(SafeAvroSerialization.java:128)
> 	at org.apache.crunch.types.avro.SafeAvroSerialization$AvroWrapperSerializer.serialize(SafeAvroSerialization.java:113)
> 	at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:1135)
> 	at org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:691)
> 	at org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
> 	at org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
> 	at org.apache.crunch.impl.mr.emit.OutputEmitter.emit(OutputEmitter.java:41)
> 	at org.apache.crunch.MapFn.process(MapFn.java:34)
> 	at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:98)
> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.emit(IntermediateEmitter.java:56)
> 	at org.apache.crunch.MapFn.process(MapFn.java:34)
> 	at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:98)
> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.emit(IntermediateEmitter.java:56)
> 	at org.apache.crunch.MapFn.process(MapFn.java:34)
> 	at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:98)
> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.emit(IntermediateEmitter.java:56)
> 	at org.apache.crunch.MapFn.process(MapFn.java:34)
> 	at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:98)
> 	at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:109)
> 	at org.apache.crunch.impl.mr.run.CrunchMapper.map(CrunchMapper.java:60)
> 	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
> 	at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
> 	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:339)
> 	at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:162)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:396)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1491)
> 	at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:157)
> Either implicit namespace "crunch" should not be added anywhere. Or it must be added,
if no namespace provided, when reading schema from the avro file i At.java:
>   public static SourceTarget<GenericData.Record> avroFile(Path path, Configuration
conf) {
>     return avroFile(path, Avros.generics(From.getSchemaFromPath(path, conf)));
>   }
> I use on HEAD in master.
>  



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message