crunch-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Magnus Runesson (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CRUNCH-360) GenericData.Record avro records without schema namespace gets implicit namespace"crunch"
Date Fri, 28 Feb 2014 10:47:21 GMT

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

Magnus Runesson commented on CRUNCH-360:
----------------------------------------

Have created an example demonstrating my problem. 

Only difference between the two exampels following is second does not have namespace. And
Still crunch says it has namespace crunch.

Works:
hadoop jar demo-1.0-SNAPSHOT.jar org.linuxalert.crunch.Demo data-namespace.avro result

Not works:
hadoop jar demo-1.0-SNAPSHOT.jar org.linuxalert.crunch.Demo data-no-namespace.avro result
Gives:
java.lang.Exception: org.apache.avro.UnresolvedUnionException: Not in union [{"type":"record","name":"twitter_schema","namespace":"crunch","fields":[{"name":"username","type":"string","doc":"Name
of the user account on Twitter.com"},{"name":"tweet","type":"string","doc":"The content of
the user's Twitter message"},{"name":"timestamp","type":"long","doc":"Unix epoch time in seconds"}],"doc:":"A
basic schema for storing Twitter messages"},"null"]: {"username": "miguno", "tweet": "Rock:
Nerf paper, scissors is fine.", "timestamp": 1366150681}
	at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:403)
Caused by: org.apache.avro.UnresolvedUnionException: Not in union [{"type":"record","name":"twitter_schema","namespace":"crunch","fields":[{"name":"username","type":"string","doc":"Name
of the user account on Twitter.com"},{"name":"tweet","type":"string","doc":"The content of
the user's Twitter message"},{"name":"timestamp","type":"long","doc":"Unix epoch time in seconds"}],"doc:":"A
basic schema for storing Twitter messages"},"null"]: {"username": "miguno", "tweet": "Rock:
Nerf paper, scissors is fine.", "timestamp": 1366150681}
	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.LocalJobRunner$Job$MapTaskRunnable.run(LocalJobRunner.java:235)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:744)
1 job failure(s) occurred:
Demo: Avro(data-no-namespace.avro)+[[Extract key from input 1+j... (1/1)(1): Job failed!

> 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.1.5#6160)

Mime
View raw message