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

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

I have a fix that seem to work at: 
https://github.com/zolyfarkas/avro/commit/54728d12868f8e1b1423b494a53eb3b3bb9721bd

the fix is in Symbol.java:

{noformat}
  static void flatten(Symbol[] in, int start,
      Symbol[] out, int skip,
      Map<Sequence, Sequence> map,
      Map<Sequence, List<Fixup>> map2) {
    for (int i = start, j = skip; i < in.length; i++) {
      Symbol s = in[i].flatten(map, map2);
      if (s instanceof Sequence) {
        Symbol[] p = s.production;
        List<Fixup> l = map2.get(s);
        if (l == null) {
          System.arraycopy(p, 0, out, j, p.length);
         // Fixups need to be relocated!
          for (List<Fixup> value : map2.values()) {
              for (Fixup fixup : value) {
                  if (fixup.symbols == p) {
                      fixup.symbols = out;
                      fixup.pos += j;
                  }
              }
          }
        } else {
          l.add(new Fixup(out, j));
        }
        j += p.length;
      } else {
        out[j++] = s;
      }
    }
  }
{noformat}

please review, and let me know what you think, it would be nice to have this 
released soon...

> 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
>
> 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