You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Joe Crobak (JIRA)" <ji...@apache.org> on 2010/12/21 21:15:03 UTC

[jira] Created: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

SpecificDatumWriter is using GenericData.resolveUnion sometimes
---------------------------------------------------------------

                 Key: AVRO-717
                 URL: https://issues.apache.org/jira/browse/AVRO-717
             Project: Avro
          Issue Type: Bug
          Components: java
    Affects Versions: 1.4.1
         Environment: Mac, Java 6.
            Reporter: Joe Crobak


From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]

I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
{code}
   public void testDataWriteWithSchema() throws IOException {
     final DataFileWriter<Event writer =
       new DataFileWriter<Event>(new SpecificDatumWriter<Event>());

     writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
     writer.append(getEvent());
     writer.close();
   }

   public void testDataWriteWithSchemaWithClass() throws IOException {
     final DataFileWriter<Event writer =
       new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));

     writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
     writer.append(getEvent());
     writer.close();
   }
{code}


 When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
{noformat}
 Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
 at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
 at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
 at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
 at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
 at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
 at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
{noformat}

Scott Carey's reply:

{quote}
This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
instead of SpecificData.resolveUnion(). 

The empty param constructor in SpecificDatumWriter should be
public SpecificDatumWriter() \{ super(SpecificData.get()); }


public SpecificDatumWriter() { }
{quote}


-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-717:
------------------------------

      Resolution: Fixed
    Hadoop Flags: [Reviewed]
          Status: Resolved  (was: Patch Available)

I committed this.  Thanks, Joe!

> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>            Assignee: Joe Crobak
>             Fix For: 1.5.0
>
>         Attachments: AVRO-717.patch, AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Joe Crobak (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Joe Crobak updated AVRO-717:
----------------------------

    Attachment: AVRO-717.patch

Updated patch, with a test-case.

Changes are:
 * Fix the default constructor for SpecificDatumWriter to fix resolveUnion.
 * Add a test for resolveUnion using a new test record type.
 * Update existing test that assumes 5 items in simple.avpr



> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>            Assignee: Joe Crobak
>             Fix For: 1.5.0
>
>         Attachments: AVRO-717.patch, AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-717:
------------------------------

    Fix Version/s: 1.5.0
         Assignee: Joe Crobak

Scott, where should we put a test for this under the new layout?

> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>            Assignee: Joe Crobak
>             Fix For: 1.5.0
>
>         Attachments: AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Joe Crobak (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Joe Crobak updated AVRO-717:
----------------------------

    Attachment: AVRO-717.patch

Patch to fix the default constructor of SpecificDatumWriter per Scott's suggestion.

Given the nature of the lang/java/avro/ source-tree, it's non-trivial to add a specific data test.  I can't use the avro-maven plugin to generate sources because this would introduce a circular dependency between modules.

I have verified the fix, though, with a down-stream project.  If there's an appropriate module to place this test, I can do that, too.

> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>         Attachments: AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Joe Crobak (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Joe Crobak updated AVRO-717:
----------------------------

    Status: Patch Available  (was: Open)

> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>            Assignee: Joe Crobak
>             Fix For: 1.5.0
>
>         Attachments: AVRO-717.patch, AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12975876#action_12975876 ] 

Scott Carey commented on AVRO-717:
----------------------------------

Add it  under the ipc sub-project.  Perhaps as part of org/apache/avro/specific.   Most testing is in ipc, because we have some dependencies that are a legacy from the old single-project structure entangling tests and test utilities from various projects.   I want to clean that up some later, but for now its just fine to put most testing code in ipc -- we build and test it all together as a unit.

The easiest thing to do is use the SpecificRecord classes that the ipc build already generates, such as org.apache.avro.test.Message.java.    Or, from ipc the avro-maven-plugin is available, or SpecificCompiler can be called directly like TestSpecificCompiler.



> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>            Assignee: Joe Crobak
>             Fix For: 1.5.0
>
>         Attachments: AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Joe Crobak (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Joe Crobak updated AVRO-717:
----------------------------

    Status: Patch Available  (was: Open)

> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>         Attachments: AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (AVRO-717) SpecificDatumWriter is using GenericData.resolveUnion sometimes

Posted by "Joe Crobak (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-717?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Joe Crobak updated AVRO-717:
----------------------------

    Status: Open  (was: Patch Available)

Sounds good.  I'll get together an updated patch with a test in the ipc module.

> SpecificDatumWriter is using GenericData.resolveUnion sometimes
> ---------------------------------------------------------------
>
>                 Key: AVRO-717
>                 URL: https://issues.apache.org/jira/browse/AVRO-717
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.4.1
>         Environment: Mac, Java 6.
>            Reporter: Joe Crobak
>            Assignee: Joe Crobak
>             Fix For: 1.5.0
>
>         Attachments: AVRO-717.patch
>
>
> From: [avro-user|http://mail-archives.apache.org/mod_mbox/avro-user/201012.mbox/%3CAANLkTin4kEN9m-YXyRH7TLLYF-e_E2TczHca6BazO9ch@mail.gmail.com%3E]
> I've tried to use union of null and the enum-type, but I've run into an issue with this approach when using the AvroOutputFormat.  The following code summarizes my issue:
> {code}
>    public void testDataWriteWithSchema() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>());
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
>    public void testDataWriteWithSchemaWithClass() throws IOException {
>      final DataFileWriter<Event writer =
>        new DataFileWriter<Event>(new SpecificDatumWriter<Event>(Event.class));
>      writer.create(Event.SCHEMA$, new File("target/datafile-test.avro"));
>      writer.append(getEvent());
>      writer.close();
>    }
> {code}
>  When I don't pass in the Event.class to SpecificDatumWriter (the first test method), the above test fails with the following exception:
> {noformat}
>  Not in union ["null", {"type":"enum","name":"Suit","namespace":"foo","symbols":["SPADES","CLUBS","HEARS","DIAMONDS"]}]: SPADES
>  at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:382)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:67)
>  at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:100)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
>  at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:54)
>  at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:245)
> {noformat}
> Scott Carey's reply:
> {quote}
> This looks like a bug.  Can you file a ticket?  In the first case, the constructor is not
> initializing the SpecificData object, which means that it is using GenericData.resolveUnion()
> instead of SpecificData.resolveUnion(). 
> The empty param constructor in SpecificDatumWriter should be
> public SpecificDatumWriter() \{ super(SpecificData.get()); }
> public SpecificDatumWriter() { }
> {quote}

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.