[ 
https://issues.apache.org/jira/browse/AVRO-4209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lari Hotari updated AVRO-4209:
------------------------------
    Description: 
{{ After AVRO-3940 changes, with a recursive data structure such as }}
{code:java}
static class Node { public Node parent; } {code}
{{, the call to ReflectData.get().getSchema(Node.class) will result in a 
java.lang.StackOverflowError}}
{noformat}
java.lang.StackOverflowError
        at 
java.base/java.util.concurrent.ConcurrentHashMap.get(ConcurrentHashMap.java:936)
        at java.base/java.lang.ClassLoader.definePackage(ClassLoader.java:2125)
        at java.base/java.lang.ClassLoader.definePackage(ClassLoader.java:2108)
        at java.base/java.lang.Class.getPackage(Class.java:1132)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:713)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
.
.
.{noformat}
 

{{To reproduce the issue, add this to the TestReflectData test class:}}
{code:java}
// Add to avro/src/test/java/org/apache/avro/reflect/TestReflectData.java
@Test
void recursiveSchema() {
  // reproduce java.lang.StackOverflowError issue
  Schema schema = ReflectData.get().getSchema(Node.class);
  assertEquals(1, schema.getFields().size());
  Schema.Field field = schema.getFields().get(0);
  assertEquals("parent", field.name());
  assertEquals(schema, field.schema());
}

static class Node {
  public Node parent;
}{code}
{{I have confirmed that the test passes by reverting 
[https://github.com/apache/avro/pull/3304] changes.}}
{{Branch forked off release-1.12.1 with the test and commit 
[https://github.com/apache/avro/commit/4d62fc43afba9e195d023a101b934a3cd1db7d54]
 reverted.}}
{{(test branch diff: 
[https://github.com/apache/avro/compare/release-1.12.1...lhotari:avro:lh-reproduce-and-fix-StackOverflowError-in-1.12.1)]}}

  was:
{{ After AVRO-3940 changes, with a recursive data structure such as }}
{{}}
{code:java}
static class Node { public Node parent; } {code}
{{, the call to }}{{ReflectData.get().getSchema(Node.class) }}{{will result in 
a java.lang.StackOverflowError}}


{noformat}
java.lang.StackOverflowError
        at 
java.base/java.util.concurrent.ConcurrentHashMap.get(ConcurrentHashMap.java:936)
        at java.base/java.lang.ClassLoader.definePackage(ClassLoader.java:2125)
        at java.base/java.lang.ClassLoader.definePackage(ClassLoader.java:2108)
        at java.base/java.lang.Class.getPackage(Class.java:1132)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:713)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
        at 
org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
        at 
org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
.
.
.{noformat}
 

{{To reproduce the issue, add this to the TestReflectData test class:}}
{{}}
{code:java}
// Add to avro/src/test/java/org/apache/avro/reflect/TestReflectData.java
@Test
void recursiveSchema() {
  // reproduce java.lang.StackOverflowError issue
  Schema schema = ReflectData.get().getSchema(Node.class);
  assertEquals(1, schema.getFields().size());
  Schema.Field field = schema.getFields().get(0);
  assertEquals("parent", field.name());
  assertEquals(schema, field.schema());
}

static class Node {
  public Node parent;
}{code}

{{I have confirmed that the test passes by reverting 
[https://github.com/apache/avro/pull/3304] changes.}}
{{Branch forked off release-1.12.1 with the test and commit 
[https://github.com/apache/avro/commit/4d62fc43afba9e195d023a101b934a3cd1db7d54]
 reverted.}}
{{(test branch diff: 
[https://github.com/apache/avro/compare/release-1.12.1...lhotari:avro:lh-reproduce-and-fix-StackOverflowError-in-1.12.1)]}}


> ReflectData.getSchema fails for a POJO class that contains a field of the 
> same type
> -----------------------------------------------------------------------------------
>
>                 Key: AVRO-4209
>                 URL: https://issues.apache.org/jira/browse/AVRO-4209
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.12.1
>         Environment: Any
>            Reporter: Lari Hotari
>            Priority: Major
>
> {{ After AVRO-3940 changes, with a recursive data structure such as }}
> {code:java}
> static class Node { public Node parent; } {code}
> {{, the call to ReflectData.get().getSchema(Node.class) will result in a 
> java.lang.StackOverflowError}}
> {noformat}
> java.lang.StackOverflowError
>         at 
> java.base/java.util.concurrent.ConcurrentHashMap.get(ConcurrentHashMap.java:936)
>         at 
> java.base/java.lang.ClassLoader.definePackage(ClassLoader.java:2125)
>         at 
> java.base/java.lang.ClassLoader.definePackage(ClassLoader.java:2108)
>         at java.base/java.lang.Class.getPackage(Class.java:1132)
>         at 
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:713)
>         at 
> org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
>         at 
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
>         at 
> org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
>         at 
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
>         at 
> org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
>         at 
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
>         at 
> org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
>         at 
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
>         at 
> org.apache.avro.reflect.ReflectData.createFieldSchema(ReflectData.java:894)
>         at 
> org.apache.avro.reflect.ReflectData.createSchema(ReflectData.java:744)
> .
> .
> .{noformat}
>  
> {{To reproduce the issue, add this to the TestReflectData test class:}}
> {code:java}
> // Add to avro/src/test/java/org/apache/avro/reflect/TestReflectData.java
> @Test
> void recursiveSchema() {
>   // reproduce java.lang.StackOverflowError issue
>   Schema schema = ReflectData.get().getSchema(Node.class);
>   assertEquals(1, schema.getFields().size());
>   Schema.Field field = schema.getFields().get(0);
>   assertEquals("parent", field.name());
>   assertEquals(schema, field.schema());
> }
> static class Node {
>   public Node parent;
> }{code}
> {{I have confirmed that the test passes by reverting 
> [https://github.com/apache/avro/pull/3304] changes.}}
> {{Branch forked off release-1.12.1 with the test and commit 
> [https://github.com/apache/avro/commit/4d62fc43afba9e195d023a101b934a3cd1db7d54]
>  reverted.}}
> {{(test branch diff: 
> [https://github.com/apache/avro/compare/release-1.12.1...lhotari:avro:lh-reproduce-and-fix-StackOverflowError-in-1.12.1)]}}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to