You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Tianyu Lang (Jira)" <ji...@apache.org> on 2020/03/24 23:57:00 UTC

[jira] [Updated] (AVRO-2780) Protobuf definition with outer class causes stack overflow for ProtobufDatumReader

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

Tianyu Lang updated AVRO-2780:
------------------------------
    Description: 
We have found a bug related, but probably not limited to ProtobufDatumReader. In Protobuf, if the ".proto" file has the same name as the message, in the generated ".java" file, "OuterClass" will be appended to the outer class's name. This is documented here: [https://developers.google.com/protocol-buffers/docs/reference/java-generated#invocation]

Now we have a test protobuf class defined like so:

 
{code:java}
syntax = "proto2";

package test.avrobug;

option java_package = "com.test.protos.avrobug";
option java_generic_services = true;

enum State {
  BAD = 0;
  VERY_BAD = 1;
  SUPER_BAD = 2;
}

message Dummy {
  optional string token = 1;
  optional string description = 2;
  optional int64 count = 3;
  optional State state = 4;
}
{code}
 

 

If we first create a Protobuf object, write it to a file as Avro through ProtobufDatumReader, then read it into a Protobuf with ProtobufDatumReader, a stack overflow exception will happen. Code to reproduce is as follows:

 
{code:java}
@Test
public void ProtoToAvroOuterClassBug() throws Exception {
  DummyOuterClass.Dummy dummy = DummyOuterClass.Dummy.newBuilder()
      .setCount(50)
      .setDescription("hehe")
      .setToken("abc123")
      .setState(DummyOuterClass.State.BAD)
      .build();
  
  ProtobufDatumWriter<DummyOuterClass.Dummy> pbWriter =
      new ProtobufDatumWriter<>(DummyOuterClass.Dummy.class);
  DataFileWriter<DummyOuterClass.Dummy> dataFileWriter = new DataFileWriter<>(pbWriter);
  Schema schema = ProtobufData.get().getSchema(DummyOuterClass.Dummy.class);
  dataFileWriter.create(schema,
      new File("/Users/tianyu/Documents/DummyAvroNoDefaultValues.avro"));
  dataFileWriter.append(dummy);
  dataFileWriter.close();

  ProtobufDatumReader<DummyOuterClass.Dummy> pbReader =
      new ProtobufDatumReader<>(DummyOuterClass.Dummy.class);
  DataFileReader<DummyOuterClass.Dummy> dataFileReader =
      new DataFileReader<>(new File("/Users/tianyu/Documents/DummyAvroNoDefaultValues.avro"),
          pbReader);
  while(dataFileReader.hasNext()) {
    DummyOuterClass.Dummy record = dataFileReader.next();
    String recordStr = record.toString();
    String originStr = dummy.toString();
    System.out.println(recordStr);
    System.out.println(originStr);
  }
}
{code}
 

 

When this is run, a stack overflow exception will happen with the following stack trace:

 
{code:java}
java.lang.StackOverflowErrorjava.lang.StackOverflowError                      at java.base/java.util.concurrent.ConcurrentHashMap.get(ConcurrentHashMap.java:936)                      at org.apache.avro.specific.SpecificData.getClass(SpecificData.java:250)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:141)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:143)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:143)                      at 
..................{code}
newRecord() is recursed infinitely until stack explodes

I did a bit code tracing and I found out when Avro tries to load the corresponding Java class from the schema, it tries to load the class "com.squareup.protos.cardr.Dummy.Dummy". I suspect the double "Dummy" at the end is related to the stack overflow, but I don't have enough knowledge on Proto or Avro to know the exact reason.

 

  was:
We have found a bug with ProtobufDatumReader. In Protobuf, if the ".proto" file has the same name as the message, in the generated ".java" file, "OuterClass" will be appended to the outer class's name. This is documented here: [https://developers.google.com/protocol-buffers/docs/reference/java-generated#invocation]

Now we have a test protobuf class defined like so:

 
{code:java}
syntax = "proto2";

package test.avrobug;

option java_package = "com.test.protos.avrobug";
option java_generic_services = true;

enum State {
  BAD = 0;
  VERY_BAD = 1;
  SUPER_BAD = 2;
}

message Dummy {
  optional string token = 1;
  optional string description = 2;
  optional int64 count = 3;
  optional State state = 4;
}
{code}
 

 

If we first create a Protobuf object, write it to a file as Avro through ProtobufDatumReader, then read it into a Protobuf with ProtobufDatumReader, a stack overflow exception will happen. Code to reproduce is as follows:

 
{code:java}
@Test
public void ProtoToAvroOuterClassBug() throws Exception {
  DummyOuterClass.Dummy dummy = DummyOuterClass.Dummy.newBuilder()
      .setCount(50)
      .setDescription("hehe")
      .setToken("abc123")
      .setState(DummyOuterClass.State.BAD)
      .build();
  
  ProtobufDatumWriter<DummyOuterClass.Dummy> pbWriter =
      new ProtobufDatumWriter<>(DummyOuterClass.Dummy.class);
  DataFileWriter<DummyOuterClass.Dummy> dataFileWriter = new DataFileWriter<>(pbWriter);
  Schema schema = ProtobufData.get().getSchema(DummyOuterClass.Dummy.class);
  dataFileWriter.create(schema,
      new File("/Users/tianyu/Documents/DummyAvroNoDefaultValues.avro"));
  dataFileWriter.append(dummy);
  dataFileWriter.close();

  ProtobufDatumReader<DummyOuterClass.Dummy> pbReader =
      new ProtobufDatumReader<>(DummyOuterClass.Dummy.class);
  DataFileReader<DummyOuterClass.Dummy> dataFileReader =
      new DataFileReader<>(new File("/Users/tianyu/Documents/DummyAvroNoDefaultValues.avro"),
          pbReader);
  while(dataFileReader.hasNext()) {
    DummyOuterClass.Dummy record = dataFileReader.next();
    String recordStr = record.toString();
    String originStr = dummy.toString();
    System.out.println(recordStr);
    System.out.println(originStr);
  }
}
{code}
 

 

When this is run, a stack overflow exception will happen with the following stack trace:

 
{code:java}
java.lang.StackOverflowErrorjava.lang.StackOverflowError                      at java.base/java.util.concurrent.ConcurrentHashMap.get(ConcurrentHashMap.java:936)                      at org.apache.avro.specific.SpecificData.getClass(SpecificData.java:250)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:141)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:143)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:143)                      at 
..................{code}
newRecord() is recursed infinitely until stack explodes

I did a bit code tracing and I found out when Avro tries to load the corresponding Java class from the schema, it tries to load the class "com.squareup.protos.cardr.Dummy.Dummy". I suspect the double "Dummy" at the end is related to the stack overflow, but I don't have enough knowledge on Proto or Avro to know the exact reason.

 


> Protobuf definition with outer class causes stack overflow for ProtobufDatumReader
> ----------------------------------------------------------------------------------
>
>                 Key: AVRO-2780
>                 URL: https://issues.apache.org/jira/browse/AVRO-2780
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.9.2
>            Reporter: Tianyu Lang
>            Priority: Critical
>
> We have found a bug related, but probably not limited to ProtobufDatumReader. In Protobuf, if the ".proto" file has the same name as the message, in the generated ".java" file, "OuterClass" will be appended to the outer class's name. This is documented here: [https://developers.google.com/protocol-buffers/docs/reference/java-generated#invocation]
> Now we have a test protobuf class defined like so:
>  
> {code:java}
> syntax = "proto2";
> package test.avrobug;
> option java_package = "com.test.protos.avrobug";
> option java_generic_services = true;
> enum State {
>   BAD = 0;
>   VERY_BAD = 1;
>   SUPER_BAD = 2;
> }
> message Dummy {
>   optional string token = 1;
>   optional string description = 2;
>   optional int64 count = 3;
>   optional State state = 4;
> }
> {code}
>  
>  
> If we first create a Protobuf object, write it to a file as Avro through ProtobufDatumReader, then read it into a Protobuf with ProtobufDatumReader, a stack overflow exception will happen. Code to reproduce is as follows:
>  
> {code:java}
> @Test
> public void ProtoToAvroOuterClassBug() throws Exception {
>   DummyOuterClass.Dummy dummy = DummyOuterClass.Dummy.newBuilder()
>       .setCount(50)
>       .setDescription("hehe")
>       .setToken("abc123")
>       .setState(DummyOuterClass.State.BAD)
>       .build();
>   
>   ProtobufDatumWriter<DummyOuterClass.Dummy> pbWriter =
>       new ProtobufDatumWriter<>(DummyOuterClass.Dummy.class);
>   DataFileWriter<DummyOuterClass.Dummy> dataFileWriter = new DataFileWriter<>(pbWriter);
>   Schema schema = ProtobufData.get().getSchema(DummyOuterClass.Dummy.class);
>   dataFileWriter.create(schema,
>       new File("/Users/tianyu/Documents/DummyAvroNoDefaultValues.avro"));
>   dataFileWriter.append(dummy);
>   dataFileWriter.close();
>   ProtobufDatumReader<DummyOuterClass.Dummy> pbReader =
>       new ProtobufDatumReader<>(DummyOuterClass.Dummy.class);
>   DataFileReader<DummyOuterClass.Dummy> dataFileReader =
>       new DataFileReader<>(new File("/Users/tianyu/Documents/DummyAvroNoDefaultValues.avro"),
>           pbReader);
>   while(dataFileReader.hasNext()) {
>     DummyOuterClass.Dummy record = dataFileReader.next();
>     String recordStr = record.toString();
>     String originStr = dummy.toString();
>     System.out.println(recordStr);
>     System.out.println(originStr);
>   }
> }
> {code}
>  
>  
> When this is run, a stack overflow exception will happen with the following stack trace:
>  
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError                      at java.base/java.util.concurrent.ConcurrentHashMap.get(ConcurrentHashMap.java:936)                      at org.apache.avro.specific.SpecificData.getClass(SpecificData.java:250)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:141)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:143)                      at org.apache.avro.protobuf.ProtobufData.newRecord(ProtobufData.java:143)                      at 
> ..................{code}
> newRecord() is recursed infinitely until stack explodes
> I did a bit code tracing and I found out when Avro tries to load the corresponding Java class from the schema, it tries to load the class "com.squareup.protos.cardr.Dummy.Dummy". I suspect the double "Dummy" at the end is related to the stack overflow, but I don't have enough knowledge on Proto or Avro to know the exact reason.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)