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

Zoltan Farkas commented on AVRO-1667:
-------------------------------------

I extended, the unit test to complicate the recursiveness, an I am unable to 
break my original implementation.

I like your implementation since it does not mutate the fixups... however I 
believe in my implementation I was mutating the existing fixups, since the old 
ones were not valid fixups anymore?
I still have to re-understand this....

In any case here is the lastest unit test code, I would really like to test and 
catch the issue my original implementation had:

{code}

package org.apache.avro.io.parsing;

import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Set;
import junit.framework.Assert;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.EncoderFactory;
import org.junit.Test;


public class SymbolTest {


    private static final String SCHEMA = 
"{\"type\":\"record\",\"name\":\"SampleNode\","
            + "\"namespace\":\"org.spf4j.ssdump2.avro\",\n" +
" \"fields\":[\n" +
"    {\"name\":\"count\",\"type\":\"int\",\"default\":0},\n" +
"    {\"name\":\"otherNode\",\"type\":[\"null\",\"SampleNode\"], \"default\" : 
null},\n" +
"    {\"name\":\"subNodes\",\"type\":\n" +
"       {\"type\":\"array\",\"items\":{\n" +
"           \"type\":\"record\",\"name\":\"SamplePair\",\n" +
"           \"fields\":[\n" +
"              {\"name\":\"method\",\"type\":\n" +
"                  {\"type\":\"record\",\"name\":\"Method\",\n" +
"                  \"fields\":[\n" +
"                     
{\"name\":\"declaringClass\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},\n"
 +
"                     
{\"name\":\"methodName\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}\n"
 +
"                  ]}},\n" +
"              {\"name\":\"node\",\"type\":\"SampleNode\"},"
            + "{\"name\":\"otherNode\",\"type\":\"SampleNode\"}]}}}"
            + ",{\"name\":\"subNodes2\",\"type\":\n" +
"       {\"type\":\"array\",\"items\": \"SampleNode\"}}"
           + ",{\"name\":\"subNodes3\",\"type\":\n" +
"       {\"type\":\"array\",\"items\": \"SampleNode\"}} ]}";


    @Test
    public void testValidSymbolTree() throws IOException {

        Schema schema = new Schema.Parser().parse(SCHEMA);

        Symbol root = Symbol.root(new ResolvingGrammarGenerator()
                .generate(schema, schema, new 
HashMap<ValidatingGrammarGenerator.LitS, Symbol>()));
        validateNonNull(root, new HashSet<Symbol>());

        Schema samplePairSchema = 
schema.getField("subNodes").schema().getElementType();
        Schema methodSchema = samplePairSchema.getField("method").schema();

        GenericData.Record method = new GenericData.Record(methodSchema);
        method.put("methodName", "m1");
        method.put("declaringClass", "c1");

        GenericData.Record samplePair = new 
GenericData.Record(samplePairSchema);
        samplePair.put("method", method);

        GenericData.Record sampleNode1 = new GenericData.Record(schema);
        sampleNode1.put("subNodes", Collections.EMPTY_LIST);
        sampleNode1.put("subNodes2", Collections.EMPTY_LIST);
        sampleNode1.put("subNodes3", Collections.EMPTY_LIST);
        sampleNode1.put("count", 0);
        sampleNode1.put("otherNode", null);

        samplePair.put("node", sampleNode1);
        samplePair.put("otherNode", sampleNode1);

        GenericData.Record sampleNode2 = new GenericData.Record(schema);
        sampleNode2.put("subNodes", Arrays.asList(samplePair));
        sampleNode2.put("subNodes2", Arrays.asList(sampleNode1));
        sampleNode2.put("subNodes3", Arrays.asList(sampleNode1));
        sampleNode2.put("count", 0);
        sampleNode2.put("otherNode", null);

        ByteArrayOutputStream bos = new ByteArrayOutputStream();
        GenericDatumWriter writer = new GenericDatumWriter(schema);
        BinaryEncoder directBinaryEncoder = new 
EncoderFactory().directBinaryEncoder(bos, null);
        writer.write(sampleNode2, directBinaryEncoder);

        ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
        GenericDatumReader reader = new GenericDatumReader(schema);
        BinaryDecoder binaryDecoder = new DecoderFactory().binaryDecoder(bis, 
null);
        Object read = reader.read(null, binaryDecoder);

        System.out.println(read);
        Assert.assertEquals(read, sampleNode2);


    }

    private static void validateNonNull(final Symbol symb, Set<Symbol> seen) {
        if (seen.contains(symb)) {
            return;
        } else {
            seen.add(symb);
        }
        if (symb.production != null) {
            for (Symbol s : symb.production) {
                if (s == null) {
                    Assert.fail("invalid parsing tree should not contain 
nulls");
                }
                if (s.kind != Symbol.Kind.ROOT) {
                    validateNonNull(s, seen);;
                }
            }
        }
    }

}

{code}

> Parser symbol tree flattening is broken for recursive schemas
> -------------------------------------------------------------
>
>                 Key: AVRO-1667
>                 URL: https://issues.apache.org/jira/browse/AVRO-1667
>             Project: Avro
>          Issue Type: Bug
>    Affects Versions: 1.7.7
>            Reporter: Zoltan Farkas
>         Attachments: AVRO-1667.2.patch, avro-1667.patch
>
>
> Here is a unit test to reproduce:
> {noformat}
> package org.apache.avro.io.parsing;
> import java.io.IOException;
> import java.util.HashMap;
> import java.util.HashSet;
> import java.util.Set;
> import junit.framework.Assert;
> import org.apache.avro.Schema;
> import org.junit.Test;
> public class SymbolTest {
>     private static final String SCHEMA = 
> "{\"type\":\"record\",\"name\":\"SampleNode\","
>             + "\"namespace\":\"org.spf4j.ssdump2.avro\",\n" +
> " \"fields\":[\n" +
> "    {\"name\":\"count\",\"type\":\"int\",\"default\":0},\n" +
> "    {\"name\":\"subNodes\",\"type\":\n" +
> "       {\"type\":\"array\",\"items\":{\n" +
> "           \"type\":\"record\",\"name\":\"SamplePair\",\n" +
> "           \"fields\":[\n" +
> "              {\"name\":\"method\",\"type\":\n" +
> "                  {\"type\":\"record\",\"name\":\"Method\",\n" +
> "                  \"fields\":[\n" +
> "                     
> {\"name\":\"declaringClass\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},\n"
>  +
> "                     
> {\"name\":\"methodName\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}\n"
>  +
> "                  ]}},\n" +
> "              {\"name\":\"node\",\"type\":\"SampleNode\"}]}}}]}";
>     @Test
>     public void testSomeMethod() throws IOException {
>         Schema schema = new Schema.Parser().parse(SCHEMA);
>         Symbol root = Symbol.root(new ResolvingGrammarGenerator()
>                 .generate(schema, schema, new 
> HashMap<ValidatingGrammarGenerator.LitS, Symbol>()));
>         validateNonNull(root, new HashSet<Symbol>());
>     }
>     private static void validateNonNull(final Symbol symb, Set<Symbol> seen) {
>         if (seen.contains(symb)) {
>             return;
>         } else {
>             seen.add(symb);
>         }
>         if (symb.production != null) {
>             for (Symbol s : symb.production) {
>                 if (s == null) {
>                     Assert.fail("invalid parsing tree should not contain 
> nulls");
>                 }
>                 if (s.kind != Symbol.Kind.ROOT) {
>                     validateNonNull(s, seen);;
>                 }
>             }
>         }
>     }
> }
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to