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

ASF GitHub Bot commented on AVRO-2274:
--------------------------------------

rstata closed pull request #393: AVRO-2274 Improve resolving performance when 
schemas don't change.
URL: https://github.com/apache/avro/pull/393
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java 
b/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java
index 8f1f6a95b..45ff922fd 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/io/ResolvingDecoder.java
@@ -129,6 +129,19 @@ public static Object resolve(Schema writer, Schema reader)
       fields;
   }
 
+  /**
+   * Same as {@link readFieldOrder} except that it returns
+   * <tt>null</tt> if there was no reordering of fields, i.e., if the
+   * correct thing for the reader to do is to read (all) of its fields
+   * in the order specified by its own schema (useful for
+   * optimizations).
+   */
+  public final Schema.Field[] readFieldOrderIfDiff() throws IOException {
+    Symbol.FieldOrderAction top
+      = (Symbol.FieldOrderAction) parser.advance(Symbol.FIELD_ACTION);
+    return (top.noReorder ? null : top.fields);
+  }
+
   /**
    * Consume any more data that has been written by the writer but not
    * needed by the reader so that the the underlying decoder is in proper
@@ -252,6 +265,7 @@ public int readEnum() throws IOException {
     parser.advance(Symbol.ENUM);
     Symbol.EnumAdjustAction top = (Symbol.EnumAdjustAction) parser.popSymbol();
     int n = in.readEnum();
+    if (top.noAdjustments) return n;
     Object o = top.adjustments[n];
     if (o instanceof Integer) {
       return ((Integer) o).intValue();
@@ -263,9 +277,17 @@ public int readEnum() throws IOException {
   @Override
   public int readIndex() throws IOException {
     parser.advance(Symbol.UNION);
-    Symbol.UnionAdjustAction top = (Symbol.UnionAdjustAction) 
parser.popSymbol();
-    parser.pushSymbol(top.symToParse);
-    return top.rindex;
+    Symbol top = parser.popSymbol();
+    int result;
+    if (top instanceof Symbol.UnionAdjustAction) {
+      result = ((Symbol.UnionAdjustAction) top).rindex;
+      top = ((Symbol.UnionAdjustAction) top).symToParse;
+    } else {
+      result = in.readIndex();
+      top = ((Symbol.Alternative) top).getSymbol(result);
+    }
+    parser.pushSymbol(top);
+    return result;
   }
 
   @Override
diff --git 
a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
 
b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
index 71978824b..61073dce8 100644
--- 
a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
+++ 
b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/ResolvingGrammarGenerator.java
@@ -76,8 +76,8 @@ public final Symbol generate(Schema writer, Schema reader)
    * @return          The start symbol for the resolving grammar
    * @throws IOException
    */
-  public Symbol generate(Schema writer, Schema reader,
-                                Map<LitS, Symbol> seen) throws IOException
+  private Symbol generate(Schema writer, Schema reader, Map<LitS, Symbol> seen)
+    throws IOException
   {
     final Schema.Type writerType = writer.getType();
     final Schema.Type readerType = reader.getType();
@@ -204,6 +204,9 @@ public Symbol generate(Schema writer, Schema reader,
 
   private Symbol resolveUnion(Schema writer, Schema reader,
       Map<LitS, Symbol> seen) throws IOException {
+    boolean needsAdj = ! unionEquiv(writer, reader, new HashMap<>());
+    List<Schema> alts2 = (!needsAdj ? reader.getTypes() : null);
+
     List<Schema> alts = writer.getTypes();
     final int size = alts.size();
     Symbol[] symbols = new Symbol[size];
@@ -215,12 +218,72 @@ private Symbol resolveUnion(Schema writer, Schema reader,
      */
     int i = 0;
     for (Schema w : alts) {
-      symbols[i] = generate(w, reader, seen);
+      symbols[i] = generate(w, (needsAdj ? reader : alts2.get(i)), seen);
       labels[i] = w.getFullName();
       i++;
     }
+    if (! needsAdj)
+      return Symbol.seq(Symbol.alt(symbols, labels), Symbol.UNION);
     return Symbol.seq(Symbol.alt(symbols, labels),
-                      Symbol.writerUnionAction());
+                      Symbol.WRITER_UNION_ACTION);
+  }
+
+  private static boolean unionEquiv(Schema w, Schema r, Map<LitS, Boolean> 
seen) {
+    Schema.Type wt = w.getType();
+    if (wt != r.getType()) return false;
+    if ((wt == Schema.Type.RECORD || wt == Schema.Type.FIXED || wt == 
Schema.Type.ENUM)
+        && ! (w.getFullName() == null || 
w.getFullName().equals(r.getFullName())))
+      return false;
+
+    switch (w.getType()) {
+    case NULL: case BOOLEAN: case INT: case LONG: case FLOAT: case DOUBLE:
+    case STRING: case BYTES:
+      return true;
+
+    case ARRAY: return unionEquiv(w.getElementType(), r.getElementType(), 
seen);
+    case MAP: return unionEquiv(w.getValueType(), r.getValueType(), seen);
+
+    case FIXED: return w.getFixedSize() == r.getFixedSize();
+
+    case ENUM: {
+      List<String> ws = w.getEnumSymbols();
+      List<String> rs = r.getEnumSymbols();
+      if (ws.size() != rs.size()) return false;
+      int i = 0;
+      for (i = 0; i < ws.size(); i++)
+        if (! ws.get(i).equals(rs.get(i))) break;
+      return i == ws.size();
+    }
+
+    case UNION: {
+      List<Schema> wb = w.getTypes();
+      List<Schema> rb = r.getTypes();
+      if (wb.size() != rb.size()) return false;
+      int i = 0;
+      for (i = 0; i < wb.size(); i++)
+        if (! unionEquiv(wb.get(i), rb.get(i), seen)) break;
+      return i == wb.size();
+    }
+
+    case RECORD: {
+      LitS wsc = new LitS2(w, r);
+      if (! seen.containsKey(wsc)) {
+        seen.put(wsc, true); // Be optimistic, but we may change our minds
+        List<Field> wb = w.getFields();
+        List<Field> rb = r.getFields();
+        if (wb.size() != rb.size()) seen.put(wsc, false);
+        else {
+          int i = 0;
+          for (i = 0; i < wb.size(); i++)
+            if (! unionEquiv(wb.get(i).schema(), rb.get(i).schema(), seen)) 
break;
+          seen.put(wsc, (i == wb.size()));
+        }
+      }
+      return seen.get(wsc);
+    }
+    default:
+      throw new IllegalArgumentException("Unknown schema type: " + 
w.getType());
+    }
   }
 
   private Symbol resolveRecords(Schema writer, Schema reader,
@@ -564,4 +627,3 @@ public int hashCode() {
      }
    }
 }
-
diff --git 
a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java 
b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java
index 187942400..4494ec0a2 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Symbol.java
@@ -156,6 +156,27 @@ public int flattenedSize() {
    * <tt>Sequence</tt> in the input are replaced by its production recursively.
    * Non-<tt>Sequence</tt> symbols, they internally have other symbols
    * those internal symbols also get flattened.
+   * When flattening is done, the only place there might be Sequence
+   * symbols is in the productions of a Repeater, Alternative, or the
+   * symToParse and symToSkip in a UnionAdjustAction or SkipAction.
+   *
+   * Why is this done?  We want our parsers to be fast.  If we left
+   * the grammars unflattened, then the parser would be constantly
+   * copying the contents of nested Sequence productions onto the
+   * parsing stack.  Instead, because of flattening, we have a long
+   * top-level production with no Sequences unless the Sequence is
+   * absolutely needed, e.g., in the case of a Repeater or an
+   * Alterantive.
+   *
+   * Well, this is not exactly true when recursion is involved.  Where
+   * there is a recursive record, that record will be "inlined" once,
+   * but any internal (ie, recursive) references to that record will
+   * be a Sequence for the record.  That Sequence will not further
+   * inline itself -- it will refer to itself as a Sequence.  The same
+   * is true for any records nested in this outer recursive record.
+   * Recursion is rare, and we want things to be fast in the typical
+   * case, which is why we do the flattening optimization.
+   *
    *
    * The algorithm does a few tricks to handle recursive symbol definitions.
    * In order to avoid infinite recursion with recursive symbols, we have a map
@@ -490,10 +511,20 @@ public static EnumAdjustAction enumAdjustAction(int 
rsymCount, Object[] adj) {
   }
 
   public static class EnumAdjustAction extends IntCheckAction {
+    public final boolean noAdjustments;
     public final Object[] adjustments;
     @Deprecated public EnumAdjustAction(int rsymCount, Object[] adjustments) {
       super(rsymCount);
       this.adjustments = adjustments;
+      boolean noAdj = true;
+      if (adjustments != null) {
+        int count = Math.min(rsymCount, adjustments.length);
+        noAdj = (adjustments.length <= rsymCount);
+        for (int i = 0; noAdj && i < count; i++)
+          noAdj &= ((adjustments[i] instanceof Integer)
+                    && i == (Integer)adjustments[i]);
+      }
+      this.noAdjustments = noAdj;
     }
   }
 
@@ -559,9 +590,14 @@ public static FieldOrderAction 
fieldOrderAction(Schema.Field[] fields) {
   }
 
   public static final class FieldOrderAction extends ImplicitAction {
+    public final boolean noReorder;
     public final Schema.Field[] fields;
     @Deprecated public FieldOrderAction(Schema.Field[] fields) {
       this.fields = fields;
+      boolean noReorder = true;
+      for (int i = 0; noReorder && i < fields.length; i++)
+        noReorder &= (i == fields[i].pos());
+      this.noReorder = noReorder;
     }
   }
 
@@ -645,6 +681,8 @@ public int findLabel(String l) {
   public static final Symbol MAP_END = new Symbol.Terminal("map-end");
   public static final Symbol ITEM_END = new Symbol.Terminal("item-end");
 
+  public static final Symbol WRITER_UNION_ACTION = writerUnionAction();
+
   /* a pseudo terminal used by parsers */
   public static final Symbol FIELD_ACTION =
     new Symbol.Terminal("field-action");
diff --git 
a/lang/java/avro/src/test/java/org/apache/avro/io/parsing/SymbolTest.java 
b/lang/java/avro/src/test/java/org/apache/avro/io/parsing/SymbolTest.java
index ce4d7df96..f5b9ca0b9 100644
--- a/lang/java/avro/src/test/java/org/apache/avro/io/parsing/SymbolTest.java
+++ b/lang/java/avro/src/test/java/org/apache/avro/io/parsing/SymbolTest.java
@@ -50,9 +50,7 @@
   @Test
   public void testSomeMethod() throws IOException {
     Schema schema = new Schema.Parser().parse(SCHEMA);
-
-    Symbol root = Symbol.root(new ResolvingGrammarGenerator()
-        .generate(schema, schema, new HashMap<>()));
+    Symbol root = new ResolvingGrammarGenerator().generate(schema, schema);
     validateNonNull(root, new HashSet<>());
   }
 
diff --git 
a/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
 
b/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
index 523c151fc..bef14252d 100644
--- 
a/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
+++ 
b/lang/java/compiler/src/main/velocity/org/apache/avro/compiler/specific/templates/java/classic/record.vm
@@ -555,25 +555,40 @@ public class ${this.mangle($schema.getName())}#if 
($schema.isError()) extends or
   @Override protected void customDecode(org.apache.avro.io.ResolvingDecoder in)
     throws java.io.IOException
   {
-    org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrder();
-    for (int i = 0; i < $schema.getFields().size(); i++) {
-      switch (fieldOrder[i].pos()) {
+    org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff();
+    if (fieldOrder == null) {
+## Common case: order of fields hasn't changed, so read them in a
+## fixed order according to reader's schema
+#set ($nv = 0)## Counter to ensure unique var-names
+#set ($maxnv = 0)## Holds high-water mark during recursion
+#foreach ($field in $schema.getFields())
+#set ($n = $this.mangle($field.name(), $schema.isError()))
+#set ($s = $field.schema())
+#set ($rs = "SCHEMA$.getField(""${n}"").schema()")
+#decodeVar(2 "this.${n}" $s $rs)
+
+#set ($nv = $maxnv)
+#end
+    } else {
+      for (int i = 0; i < $schema.getFields().size(); i++) {
+        switch (fieldOrder[i].pos()) {
 #set ($fieldno = 0)
 #set ($nv = 0)## Counter to ensure unique var-names
 #set ($maxnv = 0)## Holds high-water mark during recursion
 #foreach ($field in $schema.getFields())
-      case $fieldno:
+        case $fieldno:
 #set ($n = $this.mangle($field.name(), $schema.isError()))
 #set ($s = $field.schema())
 #set ($rs = "SCHEMA$.getField(""${n}"").schema()")
-#decodeVar(4 "this.${n}" $s $rs)
-        break;
+#decodeVar(6 "this.${n}" $s $rs)
+          break;
 
 #set ($nv = $maxnv)
 #set ($fieldno = $fieldno + 1)
 #end
-      default:
-        throw new java.io.IOException("Corrupt ResolvingDecoder.");
+        default:
+          throw new java.io.IOException("Corrupt ResolvingDecoder.");
+        }
       }
     }
   }
diff --git 
a/lang/java/tools/src/test/compiler/output-string/avro/examples/baseball/Player.java
 
b/lang/java/tools/src/test/compiler/output-string/avro/examples/baseball/Player.java
index c972235ed..af6e486cb 100644
--- 
a/lang/java/tools/src/test/compiler/output-string/avro/examples/baseball/Player.java
+++ 
b/lang/java/tools/src/test/compiler/output-string/avro/examples/baseball/Player.java
@@ -503,40 +503,64 @@ public Player build() {
   @Override protected void customDecode(org.apache.avro.io.ResolvingDecoder in)
     throws java.io.IOException
   {
-    org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrder();
-    for (int i = 0; i < 4; i++) {
-      switch (fieldOrder[i].pos()) {
-      case 0:
-        this.number = in.readInt();
-        break;
-
-      case 1:
-        this.first_name = in.readString();
-        break;
-
-      case 2:
-        this.last_name = in.readString();
-        break;
-
-      case 3:
-        long size0 = in.readArrayStart();
-        java.util.List<avro.examples.baseball.Position> a0 = this.position;
-        if (a0 == null) {
-          a0 = new 
SpecificData.Array<avro.examples.baseball.Position>((int)size0, 
SCHEMA$.getField("position").schema());
-          this.position = a0;
-        } else a0.clear();
-        SpecificData.Array<avro.examples.baseball.Position> ga0 = (a0 
instanceof SpecificData.Array ? 
(SpecificData.Array<avro.examples.baseball.Position>)a0 : null);
-        for ( ; 0 < size0; size0 = in.arrayNext()) {
-          for ( ; size0 != 0; size0--) {
-            avro.examples.baseball.Position e0 = (ga0 != null ? ga0.peek() : 
null);
-            e0 = avro.examples.baseball.Position.values()[in.readEnum()];
-            a0.add(e0);
-          }
+    org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff();
+    if (fieldOrder == null) {
+      this.number = in.readInt();
+
+      this.first_name = in.readString();
+
+      this.last_name = in.readString();
+
+      long size0 = in.readArrayStart();
+      java.util.List<avro.examples.baseball.Position> a0 = this.position;
+      if (a0 == null) {
+        a0 = new 
SpecificData.Array<avro.examples.baseball.Position>((int)size0, 
SCHEMA$.getField("position").schema());
+        this.position = a0;
+      } else a0.clear();
+      SpecificData.Array<avro.examples.baseball.Position> ga0 = (a0 instanceof 
SpecificData.Array ? (SpecificData.Array<avro.examples.baseball.Position>)a0 : 
null);
+      for ( ; 0 < size0; size0 = in.arrayNext()) {
+        for ( ; size0 != 0; size0--) {
+          avro.examples.baseball.Position e0 = (ga0 != null ? ga0.peek() : 
null);
+          e0 = avro.examples.baseball.Position.values()[in.readEnum()];
+          a0.add(e0);
         }
-        break;
+      }
+
+    } else {
+      for (int i = 0; i < 4; i++) {
+        switch (fieldOrder[i].pos()) {
+        case 0:
+          this.number = in.readInt();
+          break;
+
+        case 1:
+          this.first_name = in.readString();
+          break;
+
+        case 2:
+          this.last_name = in.readString();
+          break;
+
+        case 3:
+          long size0 = in.readArrayStart();
+          java.util.List<avro.examples.baseball.Position> a0 = this.position;
+          if (a0 == null) {
+            a0 = new 
SpecificData.Array<avro.examples.baseball.Position>((int)size0, 
SCHEMA$.getField("position").schema());
+            this.position = a0;
+          } else a0.clear();
+          SpecificData.Array<avro.examples.baseball.Position> ga0 = (a0 
instanceof SpecificData.Array ? 
(SpecificData.Array<avro.examples.baseball.Position>)a0 : null);
+          for ( ; 0 < size0; size0 = in.arrayNext()) {
+            for ( ; size0 != 0; size0--) {
+              avro.examples.baseball.Position e0 = (ga0 != null ? ga0.peek() : 
null);
+              e0 = avro.examples.baseball.Position.values()[in.readEnum()];
+              a0.add(e0);
+            }
+          }
+          break;
 
-      default:
-        throw new java.io.IOException("Corrupt ResolvingDecoder.");
+        default:
+          throw new java.io.IOException("Corrupt ResolvingDecoder.");
+        }
       }
     }
   }
diff --git a/lang/java/tools/src/test/compiler/output/Player.java 
b/lang/java/tools/src/test/compiler/output/Player.java
index af4e8f783..8376389f1 100644
--- a/lang/java/tools/src/test/compiler/output/Player.java
+++ b/lang/java/tools/src/test/compiler/output/Player.java
@@ -503,40 +503,64 @@ public Player build() {
   @Override protected void customDecode(org.apache.avro.io.ResolvingDecoder in)
     throws java.io.IOException
   {
-    org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrder();
-    for (int i = 0; i < 4; i++) {
-      switch (fieldOrder[i].pos()) {
-      case 0:
-        this.number = in.readInt();
-        break;
-
-      case 1:
-        this.first_name = in.readString(this.first_name instanceof Utf8 ? 
(Utf8)this.first_name : null);
-        break;
-
-      case 2:
-        this.last_name = in.readString(this.last_name instanceof Utf8 ? 
(Utf8)this.last_name : null);
-        break;
-
-      case 3:
-        long size0 = in.readArrayStart();
-        java.util.List<avro.examples.baseball.Position> a0 = this.position;
-        if (a0 == null) {
-          a0 = new 
SpecificData.Array<avro.examples.baseball.Position>((int)size0, 
SCHEMA$.getField("position").schema());
-          this.position = a0;
-        } else a0.clear();
-        SpecificData.Array<avro.examples.baseball.Position> ga0 = (a0 
instanceof SpecificData.Array ? 
(SpecificData.Array<avro.examples.baseball.Position>)a0 : null);
-        for ( ; 0 < size0; size0 = in.arrayNext()) {
-          for ( ; size0 != 0; size0--) {
-            avro.examples.baseball.Position e0 = (ga0 != null ? ga0.peek() : 
null);
-            e0 = avro.examples.baseball.Position.values()[in.readEnum()];
-            a0.add(e0);
-          }
+    org.apache.avro.Schema.Field[] fieldOrder = in.readFieldOrderIfDiff();
+    if (fieldOrder == null) {
+      this.number = in.readInt();
+
+      this.first_name = in.readString(this.first_name instanceof Utf8 ? 
(Utf8)this.first_name : null);
+
+      this.last_name = in.readString(this.last_name instanceof Utf8 ? 
(Utf8)this.last_name : null);
+
+      long size0 = in.readArrayStart();
+      java.util.List<avro.examples.baseball.Position> a0 = this.position;
+      if (a0 == null) {
+        a0 = new 
SpecificData.Array<avro.examples.baseball.Position>((int)size0, 
SCHEMA$.getField("position").schema());
+        this.position = a0;
+      } else a0.clear();
+      SpecificData.Array<avro.examples.baseball.Position> ga0 = (a0 instanceof 
SpecificData.Array ? (SpecificData.Array<avro.examples.baseball.Position>)a0 : 
null);
+      for ( ; 0 < size0; size0 = in.arrayNext()) {
+        for ( ; size0 != 0; size0--) {
+          avro.examples.baseball.Position e0 = (ga0 != null ? ga0.peek() : 
null);
+          e0 = avro.examples.baseball.Position.values()[in.readEnum()];
+          a0.add(e0);
         }
-        break;
+      }
+
+    } else {
+      for (int i = 0; i < 4; i++) {
+        switch (fieldOrder[i].pos()) {
+        case 0:
+          this.number = in.readInt();
+          break;
+
+        case 1:
+          this.first_name = in.readString(this.first_name instanceof Utf8 ? 
(Utf8)this.first_name : null);
+          break;
+
+        case 2:
+          this.last_name = in.readString(this.last_name instanceof Utf8 ? 
(Utf8)this.last_name : null);
+          break;
+
+        case 3:
+          long size0 = in.readArrayStart();
+          java.util.List<avro.examples.baseball.Position> a0 = this.position;
+          if (a0 == null) {
+            a0 = new 
SpecificData.Array<avro.examples.baseball.Position>((int)size0, 
SCHEMA$.getField("position").schema());
+            this.position = a0;
+          } else a0.clear();
+          SpecificData.Array<avro.examples.baseball.Position> ga0 = (a0 
instanceof SpecificData.Array ? 
(SpecificData.Array<avro.examples.baseball.Position>)a0 : null);
+          for ( ; 0 < size0; size0 = in.arrayNext()) {
+            for ( ; size0 != 0; size0--) {
+              avro.examples.baseball.Position e0 = (ga0 != null ? ga0.peek() : 
null);
+              e0 = avro.examples.baseball.Position.values()[in.readEnum()];
+              a0.add(e0);
+            }
+          }
+          break;
 
-      default:
-        throw new java.io.IOException("Corrupt ResolvingDecoder.");
+        default:
+          throw new java.io.IOException("Corrupt ResolvingDecoder.");
+        }
       }
     }
   }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Improve resolving performance when schemas don't change
> -------------------------------------------------------
>
>                 Key: AVRO-2274
>                 URL: https://issues.apache.org/jira/browse/AVRO-2274
>             Project: Apache Avro
>          Issue Type: Improvement
>          Components: java
>            Reporter: Raymie Stata
>            Assignee: Raymie Stata
>            Priority: Major
>
> Decoding optimizations based on the observation that schemas don't change 
> very much.  We add special-case paths to optimize the case where a 
> _sub_schema of the reader and the writer are the same.  The specific cases 
> are:
> * In the case of an enumeration, if the reader and writer are the same, then 
> we can simply return the tag written by the writer rather than "adjust" it as 
> if it might have been re-ordered.  In fact, we can do this (directly return 
> the tag written by the writer) as long as the reader-schema is an "extension" 
> of the writer's in that it may have added new symbols but hasn't renumbered 
> any of the writer's symbols.  Enumerations that either don't change at all or 
> are "extended" as defined here are the common ways to extend enumerations.  
> (Our tests show this optimization improves performance by about 3%.)
> * When the reader and writer subschemas are both unions, resolution is 
> expensive: we have an outer union preceded by a "writer-union action", but 
> each branch of this outer union consist of union-adjust actions, which are 
> heavy weight.  We optimize this case when the reader and writer unions are 
> the same: we fall back on the standard grammar used for a union, avoiding all 
> these adjustments.  Since unions are commonly used to encode "nullable" 
> fields in Avro, and nullability rarely changes as a schema evolves, this 
> optimization should help many users.  (Our tests show this optimization 
> improves performance by 25-30%, a significant win.)
> * The "custom code" generated for reading records has to read fields in a 
> loop that uses a switch statement to deal with writers that may have 
> re-ordered fields.  In most cases, however, fields have not been reordered 
> (esp. in more complex records with many record sub-schemas).  So we've added 
> a new method to ResolvingDecoder called readFieldOrderIfDiff, which is a 
> variant of the existing readFieldOrder.  If the field order has indeed 
> changed, then readFieldOrderIfDiff returns the new field order, just like 
> readFieldOrder does.  However, if the field-order hasn't changed, then 
> readFieldOrderIfDiff returns null.  We then modified the generation of 
> custom-decoders for records to add a special-case path that simply reads the 
> record's fields in order, without incurring the overhead of the loop or the 
> switch statement.  (Our tests show this optimization improves performance by 
> 8-9%, on top of the 35-40% produced by the original custom-coder 
> optimization.)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to