You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by GitBox <gi...@apache.org> on 2022/07/21 14:46:46 UTC

[GitHub] [avro] clesaec opened a new pull request, #1776: Avro 2918 polymorphism

clesaec opened a new pull request, #1776:
URL: https://github.com/apache/avro/pull/1776

   [AVRO-2918](https://issues.apache.org/jira/browse/AVRO-2918) : Proposition for polymorphism


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] opwvhk commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
opwvhk commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927516201


##########
lang/java/avro/src/test/java/org/apache/avro/io/LegacyBinaryEncoder.java:
##########
@@ -178,6 +178,11 @@ public void writeIndex(int unionIndex) throws IOException {
     encodeLong(unionIndex, out);
   }
 
+  @Override
+  public void writeExtends(int extendsIndex) throws IOException {
+    encodeLong(extendsIndex, out);

Review Comment:
   Should the extends clause be supported by the legacy encoder at all?
   
   Or should it throw?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927571179


##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java:
##########
@@ -229,8 +230,17 @@ protected AvroTypeException addAvroTypeMsg(AvroTypeException e, String s) {
    * representations.
    */
   protected void writeRecord(Schema schema, Object datum, Encoder out) throws IOException {
-    Object state = data.getRecordState(datum, schema);
-    for (Field f : schema.getFields()) {
+    final Schema realSchema;
+    if (datum instanceof GenericRecord && schema.hasChild()) {
+      realSchema = ((GenericRecord) datum).getSchema();
+    } else {
+      realSchema = schema;
+    }
+    if (!Objects.equals(realSchema, schema)) {
+      out.writeExtends(realSchema.getIndex());
+    }

Review Comment:
   Don't get this comment : 
   - schema variable represent the schema declaration (For a field or an array of)
   - realSchema variable represent the effective schema of the record (datum).
   if it's the same, no pb, we stay in classic situation, but if it's not the same, we mark it as extended type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927555030


##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -345,6 +379,9 @@ public String getFullName() {
     return getName();
   }
 
+  public void terminateInit() {

Review Comment:
   It's a kind of PostConstruct, for SchemaRecord, once all schemas has been built and list of fields set.
   But, indeed, i will see if we can do without it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r926790787


##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -964,6 +1029,48 @@
       this.hashCode = NO_HASHCODE;
     }
 
+    protected int startFieldPos() {
+      return 0;
+    }
+
+    @Override
+    public boolean hasChild() {
+      return this.childs != null && !(this.childs.isEmpty());
+    }
+
+    @Override
+    public Stream<Schema> visitHierarchy() {
+      final Stream<Schema> childsStream;
+      if (this.hasChild()) {
+        Comparator<ExtendedRecordSchema> c = Comparator.comparing(ExtendedRecordSchema::getFullName);
+        childsStream = this.childs.stream().sorted(c).flatMap((ExtendedRecordSchema e) -> e.visitHierarchy());
+      } else {
+        childsStream = Stream.empty();
+      }
+      return Stream.concat(Stream.of(this), childsStream);
+    }
+
+    public void indexHierachy() {
+      final AtomicInteger current = new AtomicInteger(0);
+      this.visitHierarchy().forEach((Schema e) -> {
+        ((RecordSchema) e).index = current.getAndIncrement();
+      });
+    }
+
+    public void terminateInit() {

Review Comment:
   ## Missing Override annotation
   
   This method overrides [Schema.terminateInit](1); it is advisable to add an Override annotation.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2783)



##########
lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java:
##########
@@ -21,44 +21,30 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
-import java.util.Collection;
+import java.util.stream.Stream;
 
 import org.apache.avro.Schema;
 import org.apache.avro.io.TestValidatingIO.Encoding;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
 
-@RunWith(Parameterized.class)
 public class TestResolvingIO {
 
-  protected final Encoding eEnc;
-  protected final int iSkipL;
-  protected final String sJsWrtSchm;
-  protected final String sWrtCls;
-  protected final String sJsRdrSchm;
-  protected final String sRdrCls;
-
-  public TestResolvingIO(Encoding encoding, int skipLevel, String jsonWriterSchema, String writerCalls,
-      String jsonReaderSchema, String readerCalls) {
-    this.eEnc = encoding;
-    this.iSkipL = skipLevel;
-    this.sJsWrtSchm = jsonWriterSchema;
-    this.sWrtCls = writerCalls;
-    this.sJsRdrSchm = jsonReaderSchema;
-    this.sRdrCls = readerCalls;
-  }
-
-  @Test
-  public void testIdentical() throws IOException {
-    performTest(eEnc, iSkipL, sJsWrtSchm, sWrtCls, sJsWrtSchm, sWrtCls);
+  @ParameterizedTest
+  @MethodSource("data2")
+  public void testIdentical(Encoding encoding, int skipLevel, String jsonWriterSchema, String writerCalls,
+      String jsonReaderSchema, String readerCalls) throws IOException {

Review Comment:
   ## Useless parameter
   
   The parameter readerCalls is unused.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2786)



##########
lang/java/avro/src/main/java/org/apache/avro/Resolver.java:
##########
@@ -805,4 +807,52 @@
       throw new IllegalArgumentException("Unknown schema type: " + write.getType());
     }
   }
+
+  /**
+   * In this case, the reader is a union and the writer is not. For this case, we
+   * need to pick the first branch of the reader that matches the writer and
+   * pretend to the reader that the index of this branch was found in the writer's
+   * data stream.
+   *
+   * To support this case, the {@link ReaderUnion} object has two (public) fields:
+   * <tt>firstMatch</tt> gives the index of the first matching branch in the
+   * reader's schema, and <tt>actualResolution</tt> is the {@link Action} that
+   * resolves the writer's schema with the schema found in the <tt>firstMatch</tt>
+   * branch of the reader's schema.
+   */
+  public static class ReaderExtends extends Action {
+    public final Schema firstMatch;
+    public final Action actualAction;
+
+    public ReaderExtends(Schema w, Schema r, GenericData d, Schema firstMatch, Action actual) {
+      super(w, r, d, Action.Type.READER_EXTENDS);
+      this.firstMatch = firstMatch;
+      this.actualAction = actual;
+    }
+
+    /**
+     * Returns a {@link ReaderUnion} action for resolving <tt>w</tt> and <tt>r</tt>,
+     * or an {@link ErrorAction} if there is no branch in the reader that matches
+     * the writer.
+     *
+     * @throws RuntimeException if <tt>r</tt> is not a union schema or <tt>w</tt>
+     *                          <em>is</em> a union schema
+     */
+    public static Action resolve(Schema w, Schema r, GenericData d, Map<SeenPair, Action> seen) {
+      if (w.getType() == Schema.Type.RECORD) {
+        throw new IllegalArgumentException("Writer schema is not Record.");
+      }
+      Schema schema = firstMatchingChild(w, r, d, seen);
+      if (schema != null) {
+        return new ReaderExtends(w, r, d, schema, Resolver.resolve(w, schema, d, seen));
+      }
+      return new ErrorAction(w, r, d, ErrorType.NO_MATCHING_BRANCH);
+    }
+
+    private static Schema firstMatchingChild(Schema w, Schema r, GenericData d, Map<SeenPair, Action> seen) {

Review Comment:
   ## Useless parameter
   
   The parameter d is unused.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2787)



##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -1059,6 +1170,69 @@
     }
   }
 
+  private static class ExtendedRecordSchema extends RecordSchema {
+
+    private final RecordSchema parent;
+
+    public ExtendedRecordSchema(RecordSchema parent, Name name, String doc, boolean isError) {
+      super(name, doc, isError);
+      this.parent = parent;
+      this.parent.addChild(this);
+    }
+
+    public ExtendedRecordSchema(RecordSchema parent, Name name, String doc, boolean isError, List<Field> fields) {
+      super(name, doc, isError, fields);
+      this.parent = parent;
+      this.parent.addChild(this);
+    }
+
+    @Override
+    public List<Field> getFields() {
+      Stream<Field> parentFields = parent.hasFields() ? parent.getFields().stream() : Stream.empty();
+      Stream<Field> currentFields = this.fields != null ? this.fields.stream() : Stream.empty();
+      return Stream.concat(parentFields, currentFields).collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean hasFields() {
+      return super.hasFields() || parent.hasFields();
+    }
+
+    @Override
+    public Field getField(String fieldname) {
+      Field f = null;
+      if (this.fields != null) {
+        f = super.getField(fieldname);
+      }
+      if (f == null) {
+        f = parent.getField(fieldname);
+      }
+      return f;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return o instanceof ExtendedRecordSchema && parent.equals(((ExtendedRecordSchema) o).parent) && super.equals(o);
+    }
+
+    @Override
+    protected String getJsonType() {
+      return "record:" + parent.name.getQualified(this.name.space);
+    }
+
+    @Override
+    public void indexHierachy() {
+      if (this.index < 0) {
+        this.parent.indexHierachy();
+      }
+    }
+
+    protected int startFieldPos() {

Review Comment:
   ## Missing Override annotation
   
   This method overrides [RecordSchema.startFieldPos](1); it is advisable to add an Override annotation.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2782)



##########
lang/java/avro/src/main/java/org/apache/avro/Resolver.java:
##########
@@ -805,4 +807,52 @@
       throw new IllegalArgumentException("Unknown schema type: " + write.getType());
     }
   }
+
+  /**
+   * In this case, the reader is a union and the writer is not. For this case, we
+   * need to pick the first branch of the reader that matches the writer and
+   * pretend to the reader that the index of this branch was found in the writer's
+   * data stream.
+   *
+   * To support this case, the {@link ReaderUnion} object has two (public) fields:
+   * <tt>firstMatch</tt> gives the index of the first matching branch in the
+   * reader's schema, and <tt>actualResolution</tt> is the {@link Action} that
+   * resolves the writer's schema with the schema found in the <tt>firstMatch</tt>
+   * branch of the reader's schema.
+   */
+  public static class ReaderExtends extends Action {
+    public final Schema firstMatch;
+    public final Action actualAction;
+
+    public ReaderExtends(Schema w, Schema r, GenericData d, Schema firstMatch, Action actual) {
+      super(w, r, d, Action.Type.READER_EXTENDS);
+      this.firstMatch = firstMatch;
+      this.actualAction = actual;
+    }
+
+    /**
+     * Returns a {@link ReaderUnion} action for resolving <tt>w</tt> and <tt>r</tt>,
+     * or an {@link ErrorAction} if there is no branch in the reader that matches
+     * the writer.
+     *
+     * @throws RuntimeException if <tt>r</tt> is not a union schema or <tt>w</tt>
+     *                          <em>is</em> a union schema
+     */
+    public static Action resolve(Schema w, Schema r, GenericData d, Map<SeenPair, Action> seen) {
+      if (w.getType() == Schema.Type.RECORD) {
+        throw new IllegalArgumentException("Writer schema is not Record.");
+      }
+      Schema schema = firstMatchingChild(w, r, d, seen);
+      if (schema != null) {
+        return new ReaderExtends(w, r, d, schema, Resolver.resolve(w, schema, d, seen));
+      }
+      return new ErrorAction(w, r, d, ErrorType.NO_MATCHING_BRANCH);
+    }
+
+    private static Schema firstMatchingChild(Schema w, Schema r, GenericData d, Map<SeenPair, Action> seen) {

Review Comment:
   ## Useless parameter
   
   The parameter seen is unused.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2788)



##########
lang/java/avro/src/test/java/org/apache/avro/io/TestResolvingIO.java:
##########
@@ -21,44 +21,30 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Arrays;
-import java.util.Collection;
+import java.util.stream.Stream;
 
 import org.apache.avro.Schema;
 import org.apache.avro.io.TestValidatingIO.Encoding;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
 
-@RunWith(Parameterized.class)
 public class TestResolvingIO {
 
-  protected final Encoding eEnc;
-  protected final int iSkipL;
-  protected final String sJsWrtSchm;
-  protected final String sWrtCls;
-  protected final String sJsRdrSchm;
-  protected final String sRdrCls;
-
-  public TestResolvingIO(Encoding encoding, int skipLevel, String jsonWriterSchema, String writerCalls,
-      String jsonReaderSchema, String readerCalls) {
-    this.eEnc = encoding;
-    this.iSkipL = skipLevel;
-    this.sJsWrtSchm = jsonWriterSchema;
-    this.sWrtCls = writerCalls;
-    this.sJsRdrSchm = jsonReaderSchema;
-    this.sRdrCls = readerCalls;
-  }
-
-  @Test
-  public void testIdentical() throws IOException {
-    performTest(eEnc, iSkipL, sJsWrtSchm, sWrtCls, sJsWrtSchm, sWrtCls);
+  @ParameterizedTest
+  @MethodSource("data2")
+  public void testIdentical(Encoding encoding, int skipLevel, String jsonWriterSchema, String writerCalls,
+      String jsonReaderSchema, String readerCalls) throws IOException {

Review Comment:
   ## Useless parameter
   
   The parameter jsonReaderSchema is unused.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2785)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #1776: AVRO-2918: Polymorphism

Posted by GitBox <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r1063401105


##########
lang/java/avro/src/main/java/org/apache/avro/io/FastReaderBuilder.java:
##########
@@ -579,6 +597,22 @@
     }
   }
 
+  public static class ExtendsReader implements FieldReader {
+    private final Schema schema;
+
+    public ExtendsReader(final Schema schema) {
+      this.schema = schema;
+    }
+
+    @Override
+    public Object read(final Object reuse, final Decoder decoder) throws IOException {
+      int index = decoder.readExtends();
+      final Schema realSchema = this.schema.findInHierachy(index);

Review Comment:
   ## Unread local variable
   
   Variable 'Schema realSchema' is never read.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/2962)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] opwvhk commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
opwvhk commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927505765


##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java:
##########
@@ -234,8 +234,16 @@ protected Object convert(Object datum, Schema schema, LogicalType type, Conversi
    * representations.
    */
   protected Object readRecord(Object old, Schema expected, ResolvingDecoder in) throws IOException {
-    final Object record = data.newRecord(old, expected);
-    final Object state = data.getRecordState(record, expected);
+    final Schema realSchema;
+    if (expected.hasChild()) {
+      int index = in.readExtends();
+      realSchema = expected.findInHierachy(index);
+    } else {
+      realSchema = expected;
+    }

Review Comment:
   This seems to be at odds with GenericDatumWriter, lines 239-241: there, the extends is optional, based on whether the parent schema or one of its children is written.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927571179


##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java:
##########
@@ -229,8 +230,17 @@ protected AvroTypeException addAvroTypeMsg(AvroTypeException e, String s) {
    * representations.
    */
   protected void writeRecord(Schema schema, Object datum, Encoder out) throws IOException {
-    Object state = data.getRecordState(datum, schema);
-    for (Field f : schema.getFields()) {
+    final Schema realSchema;
+    if (datum instanceof GenericRecord && schema.hasChild()) {
+      realSchema = ((GenericRecord) datum).getSchema();
+    } else {
+      realSchema = schema;
+    }
+    if (!Objects.equals(realSchema, schema)) {
+      out.writeExtends(realSchema.getIndex());
+    }

Review Comment:
   Don't get this comment : 
   - schema variable represent the schema declaration (For a field or an array of)
   - realSchema variable represent the effective schema of the record (datum).
   if it's the same, no pb, we stay in classic situation, but if it's not the same, we mark it as extended type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927572419


##########
lang/java/avro/src/test/java/org/apache/avro/io/LegacyBinaryEncoder.java:
##########
@@ -178,6 +178,11 @@ public void writeIndex(int unionIndex) throws IOException {
     encodeLong(unionIndex, out);
   }
 
+  @Override
+  public void writeExtends(int extendsIndex) throws IOException {
+    encodeLong(extendsIndex, out);

Review Comment:
   I don't know ... i can throw an exception ....



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] opwvhk commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
opwvhk commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927507262


##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java:
##########
@@ -229,8 +230,17 @@ protected AvroTypeException addAvroTypeMsg(AvroTypeException e, String s) {
    * representations.
    */
   protected void writeRecord(Schema schema, Object datum, Encoder out) throws IOException {
-    Object state = data.getRecordState(datum, schema);
-    for (Field f : schema.getFields()) {
+    final Schema realSchema;
+    if (datum instanceof GenericRecord && schema.hasChild()) {
+      realSchema = ((GenericRecord) datum).getSchema();
+    } else {
+      realSchema = schema;
+    }
+    if (!Objects.equals(realSchema, schema)) {
+      out.writeExtends(realSchema.getIndex());
+    }

Review Comment:
   This creates two possible binary representations of the same (parent) schema: one for writing a parent schema, and a different one for child schemata.
   
   How can the reader distinguish between the two?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] opwvhk commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
opwvhk commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927604195


##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -345,6 +379,9 @@ public String getFullName() {
     return getName();
   }
 
+  public void terminateInit() {

Review Comment:
   Ah, sort of like how the fields are marked immutable once set for the first time? (and any field use before they are set yields an exception)
   
   How about this idea:
   1. in the constructor, mark if the record schema will have children (if the parameter is omitted, it will not)
   2. have a method where you can set all child schemata, once
   
   This behaves like `setFields`: it allows us both to finalise initialisation and makes the schema unusable for (de)serialisation until the child schemata have been set (so it cannot be forgotten).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] opwvhk commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
opwvhk commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927518485


##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -345,6 +379,9 @@ public String getFullName() {
     return getName();
   }
 
+  public void terminateInit() {

Review Comment:
   What does this method do?
   
   I understand methods like `hasChild`, `visitHierarchy` and `findInHierarchy` above (though they also need javadoc), but I cannot understand the what&why of this one.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927583809


##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java:
##########
@@ -234,8 +234,16 @@ protected Object convert(Object datum, Schema schema, LogicalType type, Conversi
    * representations.
    */
   protected Object readRecord(Object old, Schema expected, ResolvingDecoder in) throws IOException {
-    final Object record = data.newRecord(old, expected);
-    final Object state = data.getRecordState(record, expected);
+    final Schema realSchema;
+    if (expected.hasChild()) {
+      int index = in.readExtends();
+      realSchema = expected.findInHierachy(index);
+    } else {
+      realSchema = expected;
+    }

Review Comment:
   indeed, i have to add unit test and fix that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927631899


##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -345,6 +379,9 @@ public String getFullName() {
     return getName();
   }
 
+  public void terminateInit() {

Review Comment:
   So, i finally find an easy way to remove this weird method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] opwvhk commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
opwvhk commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927642962


##########
lang/java/avro/src/main/java/org/apache/avro/Schema.java:
##########
@@ -345,6 +379,9 @@ public String getFullName() {
     return getName();
   }
 
+  public void terminateInit() {

Review Comment:
   I saw; lazy initialisation. Also a good idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [avro] clesaec commented on a diff in pull request #1776: Avro 2918 polymorphism

Posted by GitBox <gi...@apache.org>.
clesaec commented on code in PR #1776:
URL: https://github.com/apache/avro/pull/1776#discussion_r927632833


##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java:
##########
@@ -234,8 +234,16 @@ protected Object convert(Object datum, Schema schema, LogicalType type, Conversi
    * representations.
    */
   protected Object readRecord(Object old, Schema expected, ResolvingDecoder in) throws IOException {
-    final Object record = data.newRecord(old, expected);
-    final Object state = data.getRecordState(record, expected);
+    final Schema realSchema;
+    if (expected.hasChild()) {
+      int index = in.readExtends();
+      realSchema = expected.findInHierachy(index);
+    } else {
+      realSchema = expected;
+    }

Review Comment:
   You were right, so now writer will always write the "extends" symbol if schema has child.



##########
lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java:
##########
@@ -229,8 +230,17 @@ protected AvroTypeException addAvroTypeMsg(AvroTypeException e, String s) {
    * representations.
    */
   protected void writeRecord(Schema schema, Object datum, Encoder out) throws IOException {
-    Object state = data.getRecordState(datum, schema);
-    for (Field f : schema.getFields()) {
+    final Schema realSchema;
+    if (datum instanceof GenericRecord && schema.hasChild()) {
+      realSchema = ((GenericRecord) datum).getSchema();
+    } else {
+      realSchema = schema;
+    }
+    if (!Objects.equals(realSchema, schema)) {
+      out.writeExtends(realSchema.getIndex());
+    }

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org