avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From fo...@apache.org
Subject [avro] branch branch-1.9 updated: [AVRO-2663] Bug with nested record
Date Mon, 13 Jan 2020 11:11:40 GMT
This is an automated email from the ASF dual-hosted git repository.

fokko pushed a commit to branch branch-1.9
in repository https://gitbox.apache.org/repos/asf/avro.git


The following commit(s) were added to refs/heads/branch-1.9 by this push:
     new 248afff  [AVRO-2663] Bug with nested record
248afff is described below

commit 248afff79e7f020f1a8e9e7f7606e529d3107326
Author: Fokko Driesprong <fokko@apache.org>
AuthorDate: Mon Dec 23 13:13:50 2019 +0100

    [AVRO-2663] Bug with nested record
---
 .../src/main/java/org/apache/avro/Resolver.java    | 173 ++++++++++++---------
 .../java/org/apache/avro/generic/GenericData.java  |  11 +-
 .../apache/avro/generic/GenericDatumReader.java    |  51 +++---
 .../java/org/apache/avro/io/BinaryDecoder.java     |   2 +-
 .../java/org/apache/avro/io/ResolvingDecoder.java  |  13 +-
 .../java/org/apache/avro/io/parsing/Parser.java    |  10 +-
 .../avro/io/parsing/ResolvingGrammarGenerator.java |  32 ++--
 .../java/org/apache/avro/reflect/ReflectData.java  |  10 +-
 .../apache/avro/reflect/ReflectDatumReader.java    |  23 +--
 .../apache/avro/specific/SpecificDatumReader.java  |  14 +-
 .../apache/avro/util/internal/JacksonUtils.java    |  14 +-
 .../org/apache/avro/TestSchemaCompatibility.java   | 113 +++++++++++---
 .../src/test/resources/schema-location-read.json   |  28 ++++
 .../src/test/resources/schema-location-write.json  |  28 ++++
 .../avro/src/test/resources/schema-location.json   |  14 ++
 .../org/apache/avro/protobuf/ProtobufData.java     |  14 +-
 .../java/org/apache/avro/thrift/ThriftData.java    |  10 +-
 17 files changed, 376 insertions(+), 184 deletions(-)

diff --git a/lang/java/avro/src/main/java/org/apache/avro/Resolver.java b/lang/java/avro/src/main/java/org/apache/avro/Resolver.java
index 16a88a9..fead519 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/Resolver.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/Resolver.java
@@ -73,8 +73,9 @@ public class Resolver {
     final Schema.Type wType = w.getType();
     final Schema.Type rType = r.getType();
 
-    if (wType == Schema.Type.UNION)
+    if (wType == Schema.Type.UNION) {
       return WriterUnion.resolve(w, r, d, seen);
+    }
 
     if (wType == rType) {
       switch (wType) {
@@ -89,12 +90,13 @@ public class Resolver {
         return new DoNothing(w, r, d);
 
       case FIXED:
-        if (w.getFullName() != null && !w.getFullName().equals(r.getFullName()))
+        if (w.getFullName() != null && !w.getFullName().equals(r.getFullName())) {
           return new ErrorAction(w, r, d, ErrorType.NAMES_DONT_MATCH);
-        else if (w.getFixedSize() != r.getFixedSize())
+        } else if (w.getFixedSize() != r.getFixedSize()) {
           return new ErrorAction(w, r, d, ErrorType.SIZES_DONT_MATCH);
-        else
+        } else {
           return new DoNothing(w, r, d);
+        }
 
       case ARRAY:
         Action et = resolve(w.getElementType(), r.getElementType(), d, seen);
@@ -113,10 +115,11 @@ public class Resolver {
       default:
         throw new IllegalArgumentException("Unknown type for schema: " + wType);
       }
-    } else if (rType == Schema.Type.UNION)
+    } else if (rType == Schema.Type.UNION) {
       return ReaderUnion.resolve(w, r, d, seen);
-    else
+    } else {
       return Promote.resolve(w, r, d);
+    }
   }
 
   /**
@@ -230,9 +233,11 @@ public class Resolver {
         List<Field> wfields = writer.getFields();
         List<Field> rfields = reader.getFields();
         String fname = "<oops>";
-        for (Field rf : rfields)
-          if (writer.getField(rf.name()) == null && rf.defaultValue() == null)
+        for (Field rf : rfields) {
+          if (writer.getField(rf.name()) == null && rf.defaultValue() == null) {
             fname = rf.name();
+          }
+        }
         return ("Found " + writer.getFullName() + ", expecting " + reader.getFullName() + ", missing required field "
             + fname);
       }
@@ -255,7 +260,7 @@ public class Resolver {
 
     /**
      * Return a promotion.
-     * 
+     *
      * @param w Writer's schema
      * @param r Rearder's schema
      * @return a {@link Promote} schema if the two schemas are compatible, or
@@ -264,10 +269,11 @@ public class Resolver {
      *                                  not different.
      */
     public static Action resolve(Schema w, Schema r, GenericData d) {
-      if (isValid(w, r))
+      if (isValid(w, r)) {
         return new Promote(w, r, d);
-      else
+      } else {
         return new ErrorAction(w, r, d, ErrorType.INCOMPATIBLE_SCHEMA_TYPES);
+      }
     }
 
     /**
@@ -368,8 +374,9 @@ public class Resolver {
       int rsymCount = r.getEnumSymbols().size();
       int count = Math.min(rsymCount, adj.length);
       noAdj = (adj.length <= rsymCount);
-      for (int i = 0; noAdj && i < count; i++)
+      for (int i = 0; noAdj && i < count; i++) {
         noAdj &= (i == adj[i]);
+      }
       this.noAdjustmentsNeeded = noAdj;
     }
 
@@ -454,8 +461,9 @@ public class Resolver {
      */
     public boolean noReorder() {
       boolean result = true;
-      for (int i = 0; result && i < readerOrder.length; i++)
+      for (int i = 0; result && i < readerOrder.length; i++) {
         result &= (i == readerOrder[i].pos());
+      }
       return result;
     }
 
@@ -472,14 +480,15 @@ public class Resolver {
      * if there was a problem resolving. An {@link ErrorAction} is returned when
      * either the two record-schemas don't have the same name, or if the writer is
      * missing a field for which the reader does not have a default value.
-     * 
+     *
      * @throws RuntimeException if writer and reader schemas are not both records
      */
-    static Action resolve(Schema w, Schema r, GenericData d, Map<SeenPair, Action> seen) {
-      SeenPair wr = new SeenPair(w, r);
-      Action result = seen.get(wr);
-      if (result != null)
+    static Action resolve(Schema writeSchema, Schema readSchema, GenericData data, Map<SeenPair, Action> seen) {
+      final SeenPair writeReadPair = new SeenPair(writeSchema, readSchema);
+      Action result = seen.get(writeReadPair);
+      if (result != null) {
         return result;
+      }
 
       /*
        * Current implementation doesn't do this check. To pass regressions tests, we
@@ -487,39 +496,48 @@ public class Resolver {
        * w.getFullName().equals(r.getFullName())) { result = new ErrorAction(w, r, d,
        * ErrorType.NAMES_DONT_MATCH); seen.put(wr, result); return result; }
        */
-      List<Field> wfields = w.getFields();
-      List<Field> rfields = r.getFields();
+      final List<Field> writeFields = writeSchema.getFields();
+      final List<Field> readFields = readSchema.getFields();
 
       int firstDefault = 0;
-      for (Schema.Field wf : wfields)
-        if (r.getField(wf.name()) != null)
-          firstDefault++;
-      Action[] actions = new Action[wfields.size()];
-      Field[] reordered = new Field[rfields.size()];
-      Object[] defaults = new Object[reordered.length - firstDefault];
-      result = new RecordAdjust(w, r, d, actions, reordered, firstDefault, defaults);
-      seen.put(wr, result); // Insert early to handle recursion
+      for (Schema.Field writeField : writeFields) {
+        // The writeFields that are also in the readschema
+        if (readSchema.getField(writeField.name()) != null) {
+          ++firstDefault;
+        }
+      }
+      final Action[] actions = new Action[writeFields.size()];
+      final Field[] reordered = new Field[readFields.size()];
+      final Object[] defaults = new Object[reordered.length - firstDefault];
+      result = new RecordAdjust(writeSchema, readSchema, data, actions, reordered, firstDefault, defaults);
+      seen.put(writeReadPair, result); // Insert early to handle recursion
 
       int i = 0;
       int ridx = 0;
-      for (Field wField : wfields) {
-        Field rField = r.getField(wField.name());
-        if (rField != null) {
-          reordered[ridx++] = rField;
-          actions[i++] = Resolver.resolve(wField.schema(), rField.schema(), d, seen);
-        } else
-          actions[i++] = new Skip(wField.schema(), d);
+      for (Field writeField : writeFields) {
+        final Field readField = readSchema.getField(writeField.name());
+        if (readField != null) {
+          reordered[ridx++] = readField;
+          actions[i++] = Resolver.resolve(writeField.schema(), readField.schema(), data, seen);
+        } else {
+          actions[i++] = new Skip(writeField.schema(), data);
+        }
       }
-      for (Field rf : rfields)
-        if (w.getField(rf.name()) == null)
-          if (rf.defaultValue() == null) {
-            result = new ErrorAction(w, r, d, ErrorType.MISSING_REQUIRED_FIELD);
-            seen.put(wr, result);
+      for (Field readField : readFields) {
+        // The field is not in the writeSchema, so we can never read it
+        // Use the default value, or throw an error otherwise
+        final Field writeField = writeSchema.getField(readField.name());
+        if (writeField == null) {
+          if (readField.defaultValue() == null) {
+            result = new ErrorAction(writeSchema, readSchema, data, ErrorType.MISSING_REQUIRED_FIELD);
+            seen.put(writeReadPair, result);
             return result;
           } else {
-            defaults[ridx - firstDefault] = d.getDefaultValue(rf);
-            reordered[ridx++] = rf;
+            defaults[ridx - firstDefault] = data.getDefaultValue(readField);
+            reordered[ridx++] = readField;
           }
+        }
+      }
       return result;
     }
   }
@@ -546,15 +564,16 @@ public class Resolver {
       actions = a;
     }
 
-    public static Action resolve(Schema w, Schema r, GenericData d, Map<SeenPair, Action> seen) {
-      boolean ueqv = unionEquiv(w, r, new HashMap<>());
-      List<Schema> wb = w.getTypes();
-      List<Schema> rb = (ueqv ? r.getTypes() : null);
+    public static Action resolve(Schema writeSchema, Schema readSchema, GenericData data, Map<SeenPair, Action> seen) {
+      boolean ueqv = unionEquiv(writeSchema, readSchema, new HashMap<>());
+      final List<Schema> wb = writeSchema.getTypes();
+      final List<Schema> rb = (ueqv ? readSchema.getTypes() : null);
       int sz = wb.size();
-      Action[] actions = new Action[sz];
-      for (int i = 0; i < sz; i++)
-        actions[i] = Resolver.resolve(wb.get(i), (ueqv ? rb.get(i) : r), d, seen);
-      return new WriterUnion(w, r, d, ueqv, actions);
+      final Action[] actions = new Action[sz];
+      for (int i = 0; i < sz; i++) {
+        actions[i] = Resolver.resolve(wb.get(i), (ueqv ? rb.get(i) : readSchema), data, seen);
+      }
+      return new WriterUnion(writeSchema, readSchema, data, ueqv, actions);
     }
   }
 
@@ -584,16 +603,18 @@ public class Resolver {
      * 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.UNION)
+      if (w.getType() == Schema.Type.UNION) {
         throw new IllegalArgumentException("Writer schema is union.");
+      }
       int i = firstMatchingBranch(w, r, d, seen);
-      if (0 <= i)
+      if (0 <= i) {
         return new ReaderUnion(w, r, d, i, Resolver.resolve(w, r.getTypes().get(i), d, seen));
+      }
       return new ErrorAction(w, r, d, ErrorType.NO_MATCHING_BRANCH);
     }
 
@@ -608,7 +629,7 @@ public class Resolver {
       int j = 0;
       int structureMatch = -1;
       for (Schema b : r.getTypes()) {
-        if (vt == b.getType())
+        if (vt == b.getType()) {
           if (vt == Schema.Type.RECORD || vt == Schema.Type.ENUM || vt == Schema.Type.FIXED) {
             String vname = w.getFullName();
             String bname = b.getFullName();
@@ -624,14 +645,17 @@ public class Resolver {
                 structureMatch = j;
               }
             }
-          } else
+          } else {
             return j;
+          }
+        }
         j++;
       }
 
       // if there is a record structure match, return it
-      if (structureMatch >= 0)
+      if (structureMatch >= 0) {
         return structureMatch;
+      }
 
       // then scan match via numeric promotion
       j = 0;
@@ -680,9 +704,11 @@ public class Resolver {
       if (action instanceof ErrorAction)
         return true;
       else
-        for (Action a : ((RecordAdjust) action).fieldActions)
-          if (a instanceof ErrorAction)
+        for (Action a : ((RecordAdjust) action).fieldActions) {
+          if (a instanceof ErrorAction) {
             return true;
+          }
+        }
       return false;
     }
   }
@@ -696,8 +722,9 @@ public class Resolver {
     // getName should be used here. Using name rather than fully qualified name
     // maintains backwards compatibility.
     if ((wt == Schema.Type.RECORD || wt == Schema.Type.FIXED || wt == Schema.Type.ENUM)
-        && !(w.getName() == null || w.getName().equals(r.getName())))
+        && !(w.getName() == null || w.getName().equals(r.getName()))) {
       return false;
+    }
 
     switch (w.getType()) {
     case NULL:
@@ -723,10 +750,12 @@ public class Resolver {
       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)))
+      int i;
+      for (i = 0; i < ws.size(); i++) {
+        if (!ws.get(i).equals(rs.get(i))) {
           break;
+        }
+      }
       return i == ws.size();
     }
 
@@ -735,10 +764,12 @@ public class Resolver {
       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))
+      int i;
+      for (i = 0; i < wb.size(); i++) {
+        if (!unionEquiv(wb.get(i), rb.get(i), seen)) {
           break;
+        }
+      }
       return i == wb.size();
     }
 
@@ -748,13 +779,15 @@ public class Resolver {
         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())
+        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))
+        } else {
+          int i;
+          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()));
         }
       }
diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
index 2d15262..1aa50ac 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericData.java
@@ -791,8 +791,8 @@ public class GenericData {
    * to a record instance. The default implementation is for
    * {@link IndexedRecord}.
    */
-  public void setField(Object record, String name, int position, Object o) {
-    ((IndexedRecord) record).put(position, o);
+  public void setField(Object record, String name, int position, Object value) {
+    ((IndexedRecord) record).put(position, value);
   }
 
   /**
@@ -814,8 +814,8 @@ public class GenericData {
   }
 
   /** Version of {@link #setField} that has state. */
-  protected void setField(Object r, String n, int p, Object o, Object state) {
-    setField(r, n, p, o);
+  protected void setField(Object record, String name, int position, Object value, Object state) {
+    setField(record, name, position, value);
   }
 
   /** Version of {@link #getField} that has state. */
@@ -848,8 +848,9 @@ public class GenericData {
     }
 
     Integer i = union.getIndexNamed(getSchemaName(datum));
-    if (i != null)
+    if (i != null) {
       return i;
+    }
     throw new UnresolvedUnionException(union, datum);
   }
 
diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
index d555400..89df601 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumReader.java
@@ -18,13 +18,14 @@
 package org.apache.avro.generic;
 
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import java.util.Collection;
 import java.nio.ByteBuffer;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.avro.AvroRuntimeException;
 import org.apache.avro.Conversion;
@@ -203,7 +204,7 @@ public class GenericDatumReader<D> implements DatumReader<D> {
   /**
    * Convert a underlying representation of a logical type (such as a ByteBuffer)
    * to a higher level object (such as a BigDecimal).
-   * 
+   *
    * @throws IllegalArgumentException if a null schema or logicalType is passed in
    *                                  while datum and conversion are not null.
    *                                  Please be noticed that the exception type
@@ -224,28 +225,36 @@ public class GenericDatumReader<D> implements DatumReader<D> {
    * representations.
    */
   protected Object readRecord(Object old, Schema expected, ResolvingDecoder in) throws IOException {
-    Object r = data.newRecord(old, expected);
-    Object state = data.getRecordState(r, expected);
+    final Object record = data.newRecord(old, expected);
+    final Object state = data.getRecordState(record, expected);
+    final List<Field> expectedFields = expected.getFields();
 
-    for (Field f : in.readFieldOrder()) {
-      int pos = f.pos();
-      String name = f.name();
+    for (Field field : in.readFieldOrder()) {
+      int pos = field.pos();
+      String name = field.name();
       Object oldDatum = null;
       if (old != null) {
-        oldDatum = data.getField(r, name, pos, state);
+        oldDatum = data.getField(record, name, pos, state);
+      }
+
+      readField(record, field, oldDatum, in, state);
+
+      // In case the expected field isn't in the read field
+      if (!expectedFields.get(pos).equals(field)) {
+        data.setField(record, field.name(), field.pos(), data.getDefaultValue(expectedFields.get(pos)));
       }
-      readField(r, f, oldDatum, in, state);
     }
 
-    return r;
+    return record;
   }
 
   /**
    * Called to read a single field of a record. May be overridden for more
    * efficient or alternate implementations.
    */
-  protected void readField(Object r, Field f, Object oldDatum, ResolvingDecoder in, Object state) throws IOException {
-    data.setField(r, f.name(), f.pos(), read(oldDatum, f.schema(), in), state);
+  protected void readField(Object record, Field field, Object oldDatum, ResolvingDecoder in, Object state)
+      throws IOException {
+    data.setField(record, field.name(), field.pos(), read(oldDatum, field.schema(), in), state);
   }
 
   /**
@@ -379,7 +388,7 @@ public class GenericDatumReader<D> implements DatumReader<D> {
   /**
    * Called to create an fixed value. May be overridden for alternate fixed
    * representations. By default, returns {@link GenericFixed}.
-   * 
+   *
    * @deprecated As of Avro 1.6.0 this method has been moved to
    *             {@link GenericData#createFixed(Object, Schema)}
    */
@@ -391,7 +400,7 @@ public class GenericDatumReader<D> implements DatumReader<D> {
   /**
    * Called to create an fixed value. May be overridden for alternate fixed
    * representations. By default, returns {@link GenericFixed}.
-   * 
+   *
    * @deprecated As of Avro 1.6.0 this method has been moved to
    *             {@link GenericData#createFixed(Object, byte[], Schema)}
    */
@@ -407,7 +416,7 @@ public class GenericDatumReader<D> implements DatumReader<D> {
    * they should either be removed from the old object, or it should create a new
    * instance that conforms to the schema. By default, this returns a
    * {@link GenericData.Record}.
-   * 
+   *
    * @deprecated As of Avro 1.6.0 this method has been moved to
    *             {@link GenericData#newRecord(Object, Schema)}
    */
@@ -452,10 +461,12 @@ public class GenericDatumReader<D> implements DatumReader<D> {
    */
   protected Object readString(Object old, Schema expected, Decoder in) throws IOException {
     Class stringClass = getStringClass(expected);
-    if (stringClass == String.class)
+    if (stringClass == String.class) {
       return in.readString();
-    if (stringClass == CharSequence.class)
+    }
+    if (stringClass == CharSequence.class) {
       return readString(old, in);
+    }
     return newInstanceFromString(stringClass, in.readString());
   }
 
diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java
index 46ad2e8..c36c743 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/io/BinaryDecoder.java
@@ -306,7 +306,7 @@ public class BinaryDecoder extends Decoder {
   @Override
   public ByteBuffer readBytes(ByteBuffer old) throws IOException {
     int length = readInt();
-    ByteBuffer result;
+    final ByteBuffer result;
     if (old != null && length <= old.capacity()) {
       result = old;
       result.clear();
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 c251b23..cf225aa 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
@@ -54,7 +54,7 @@ public class ResolvingDecoder extends ValidatingDecoder {
    * Constructs a <tt>ResolvingDecoder</tt> using the given resolver. The resolver
    * must have been returned by a previous call to
    * {@link #resolve(Schema, Schema)}.
-   * 
+   *
    * @param resolver The resolver to use.
    * @param in       The underlying decoder.
    * @throws IOException
@@ -95,7 +95,7 @@ public class ResolvingDecoder extends ValidatingDecoder {
    * reader is expecting a three-field record, the first field is a long, the
    * second a string, and the third an array. In this case, a typical usage might
    * be as follows:
-   * 
+   *
    * <pre>
    *   Schema.Fields[] fieldOrder = in.readFieldOrder();
    *   for (int i = 0; i &lt; 3; i++) {
@@ -111,7 +111,7 @@ public class ResolvingDecoder extends ValidatingDecoder {
    *       break;
    *     }
    * </pre>
-   * 
+   *
    * Note that {@link ResolvingDecoder} will return only the fields expected by
    * the reader, not other fields that may have been written by the writer. Thus,
    * the iteration-count of "3" in the above loop will always be correct.
@@ -151,7 +151,7 @@ public class ResolvingDecoder extends ValidatingDecoder {
    * for the next object as well, calling this method is optional; the state of
    * this resolving decoder ensures that any leftover portions are consumed before
    * the next object is decoded.
-   * 
+   *
    * @throws IOException
    */
   public final void drain() throws IOException {
@@ -260,8 +260,9 @@ public class ResolvingDecoder extends ValidatingDecoder {
     parser.advance(Symbol.ENUM);
     Symbol.EnumAdjustAction top = (Symbol.EnumAdjustAction) parser.popSymbol();
     int n = in.readEnum();
-    if (top.noAdjustments)
+    if (top.noAdjustments) {
       return n;
+    }
     Object o = top.adjustments[n];
     if (o instanceof Integer) {
       return (Integer) o;
@@ -274,7 +275,7 @@ public class ResolvingDecoder extends ValidatingDecoder {
   public int readIndex() throws IOException {
     parser.advance(Symbol.UNION);
     Symbol top = parser.popSymbol();
-    int result;
+    final int result;
     if (top instanceof Symbol.UnionAdjustAction) {
       result = ((Symbol.UnionAdjustAction) top).rindex;
       top = ((Symbol.UnionAdjustAction) top).symToParse;
diff --git a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Parser.java b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Parser.java
index d7c0d89..12fc404 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Parser.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/io/parsing/Parser.java
@@ -36,7 +36,7 @@ public class Parser {
     /**
      * Handle the action symbol <tt>top</tt> when the <tt>input</tt> is sought to be
      * taken off the stack.
-     * 
+     *
      * @param input The input symbol from the caller of advance
      * @param top   The symbol at the top the stack.
      * @return <tt>null</tt> if advance() is to continue processing the stack. If
@@ -50,7 +50,7 @@ public class Parser {
   protected Symbol[] stack;
   protected int pos;
 
-  public Parser(Symbol root, ActionHandler symbolHandler) throws IOException {
+  public Parser(Symbol root, ActionHandler symbolHandler) {
     this.symbolHandler = symbolHandler;
     this.stack = new Symbol[5]; // Start small to make sure expansion code works
     this.stack[0] = root;
@@ -68,7 +68,7 @@ public class Parser {
    * Recursively replaces the symbol at the top of the stack with its production,
    * until the top is a terminal. Then checks if the top symbol matches the
    * terminal symbol suppled <tt>terminal</tt>.
-   * 
+   *
    * @param input The symbol to match against the terminal at the top of the
    *              stack.
    * @return The terminal symbol at the top of the stack unless an implicit action
@@ -101,7 +101,7 @@ public class Parser {
    * Performs any implicit actions at the top the stack, expanding any production
    * (other than the root) that may be encountered. This method will fail if there
    * are any repeaters on the stack.
-   * 
+   *
    * @throws IOException
    */
   public final void processImplicitActions() throws IOException {
@@ -138,7 +138,7 @@ public class Parser {
    * Pushes the production for the given symbol <tt>sym</tt>. If <tt>sym</tt> is a
    * repeater and <tt>input</tt> is either {@link Symbol#ARRAY_END} or
    * {@link Symbol#MAP_END} pushes nothing.
-   * 
+   *
    * @param sym
    */
   public final void pushProduction(Symbol sym) {
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 7f0ee35..fd02b46 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
@@ -54,7 +54,7 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
   /**
    * Resolves the writer schema <tt>writer</tt> and the reader schema
    * <tt>reader</tt> and returns the start symbol for the grammar generated.
-   * 
+   *
    * @param writer The schema used by the writer
    * @param reader The schema used by the reader
    * @return The start symbol for the resolving grammar
@@ -105,8 +105,9 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
       return Symbol.seq(Symbol.repeat(Symbol.MAP_END, es, Symbol.STRING), Symbol.MAP_START);
 
     } else if (action.writer.getType() == Schema.Type.UNION) {
-      if (((Resolver.WriterUnion) action).unionEquiv)
+      if (((Resolver.WriterUnion) action).unionEquiv) {
         return simpleGen(action.writer, seen);
+      }
       Resolver.Action[] branches = ((Resolver.WriterUnion) action).actions;
       Symbol[] symbols = new Symbol[branches.length];
       String[] labels = new String[branches.length];
@@ -120,9 +121,10 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
     } else if (action instanceof Resolver.EnumAdjust) {
       Resolver.EnumAdjust e = (Resolver.EnumAdjust) action;
       Object[] adjs = new Object[e.adjustments.length];
-      for (int i = 0; i < adjs.length; i++)
+      for (int i = 0; i < adjs.length; i++) {
         adjs[i] = (0 <= e.adjustments[i] ? new Integer(e.adjustments[i])
             : "No match for " + e.writer.getEnumSymbols().get(i));
+      }
       return Symbol.seq(Symbol.enumAdjustAction(e.reader.getEnumSymbols().size(), adjs), Symbol.ENUM);
 
     } else if (action instanceof Resolver.RecordAdjust) {
@@ -131,14 +133,17 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
         final Resolver.RecordAdjust ra = (Resolver.RecordAdjust) action;
         int defaultCount = ra.readerOrder.length - ra.firstDefault;
         int count = 1 + ra.fieldActions.length + 3 * defaultCount;
-        Symbol[] production = new Symbol[count];
+        final Symbol[] production = new Symbol[count];
         result = Symbol.seq(production);
         seen.put(action, result);
         production[--count] = Symbol.fieldOrderAction(ra.readerOrder);
-        for (Resolver.Action wfa : ra.fieldActions)
+
+        final Resolver.Action[] actions = ra.fieldActions;
+        for (Resolver.Action wfa : actions) {
           production[--count] = generate(wfa, seen);
+        }
         for (int i = ra.firstDefault; i < ra.readerOrder.length; i++) {
-          Schema.Field rf = ra.readerOrder[i];
+          final Schema.Field rf = ra.readerOrder[i];
           byte[] bb = getBinary(rf.schema(), Accessor.defaultValue(rf));
           production[--count] = Symbol.defaultStartAction(bb);
           production[--count] = simpleGen(rf.schema(), seen);
@@ -184,9 +189,9 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
           Symbol.MAP_START);
 
     case UNION: {
-      List<Schema> subs = s.getTypes();
-      Symbol[] symbols = new Symbol[subs.size()];
-      String[] labels = new String[subs.size()];
+      final List<Schema> subs = s.getTypes();
+      final Symbol[] symbols = new Symbol[subs.size()];
+      final String[] labels = new String[subs.size()];
       int i = 0;
       for (Schema b : s.getTypes()) {
         symbols[i] = simpleGen(b, seen);
@@ -198,13 +203,14 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
     case RECORD: {
       Symbol result = seen.get(s);
       if (result == null) {
-        Symbol[] production = new Symbol[s.getFields().size() + 1];
+        final Symbol[] production = new Symbol[s.getFields().size() + 1];
         result = Symbol.seq(production);
         seen.put(s, result);
         int i = production.length;
         production[--i] = Symbol.fieldOrderAction(s.getFields().toArray(new Schema.Field[0]));
-        for (Field f : s.getFields())
+        for (Field f : s.getFields()) {
           production[--i] = simpleGen(f.schema(), seen);
+        }
         // FieldOrderAction is needed even though the field-order hasn't changed,
         // because the _reader_ doesn't know the field order hasn't changed, and
         // thus it will probably call {@ ResolvingDecoder.fieldOrder} to find out.
@@ -222,7 +228,7 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
   /**
    * Returns the Avro binary encoded version of <tt>n</tt> according to the schema
    * <tt>s</tt>.
-   * 
+   *
    * @param s The schema for encoding
    * @param n The Json node that has the value to be encoded.
    * @return The binary encoded version of <tt>n</tt>.
@@ -239,7 +245,7 @@ public class ResolvingGrammarGenerator extends ValidatingGrammarGenerator {
   /**
    * Encodes the given Json node <tt>n</tt> on to the encoder <tt>e</tt> according
    * to the schema <tt>s</tt>.
-   * 
+   *
    * @param e The encoder to encode into.
    * @param s The schema for the object being encoded.
    * @param n The Json node to encode.
diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java
index 04166ee..6a7bacb 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectData.java
@@ -134,18 +134,18 @@ public class ReflectData extends SpecificData {
   }
 
   @Override
-  public void setField(Object record, String name, int position, Object o) {
-    setField(record, name, position, o, null);
+  public void setField(Object record, String name, int position, Object value) {
+    setField(record, name, position, value, null);
   }
 
   @Override
-  protected void setField(Object record, String name, int pos, Object o, Object state) {
+  protected void setField(Object record, String name, int position, Object value, Object state) {
     if (record instanceof IndexedRecord) {
-      super.setField(record, name, pos, o);
+      super.setField(record, name, position, value);
       return;
     }
     try {
-      getAccessorForField(record, name, pos, state).set(record, o);
+      getAccessorForField(record, name, position, state).set(record, value);
     } catch (IllegalAccessException | IOException e) {
       throw new AvroRuntimeException(e);
     }
diff --git a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
index 1301103..20be49e 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/reflect/ReflectDatumReader.java
@@ -254,18 +254,19 @@ public class ReflectDatumReader<T> extends SpecificDatumReader<T> {
   }
 
   @Override
-  protected void readField(Object record, Field f, Object oldDatum, ResolvingDecoder in, Object state)
+  protected void readField(Object record, Field field, Object oldDatum, ResolvingDecoder in, Object state)
       throws IOException {
     if (state != null) {
-      FieldAccessor accessor = ((FieldAccessor[]) state)[f.pos()];
+      FieldAccessor accessor = ((FieldAccessor[]) state)[field.pos()];
       if (accessor != null) {
-        if (accessor.supportsIO() && (!Schema.Type.UNION.equals(f.schema().getType()) || accessor.isCustomEncoded())) {
+        if (accessor.supportsIO()
+            && (!Schema.Type.UNION.equals(field.schema().getType()) || accessor.isCustomEncoded())) {
           accessor.read(record, in);
           return;
         }
         if (accessor.isStringable()) {
           try {
-            String asString = (String) read(null, f.schema(), in);
+            String asString = (String) read(null, field.schema(), in);
             accessor.set(record,
                 asString == null ? null : newInstanceFromString(accessor.getField().getType(), asString));
             return;
@@ -273,27 +274,27 @@ public class ReflectDatumReader<T> extends SpecificDatumReader<T> {
             throw new AvroRuntimeException("Failed to read Stringable", e);
           }
         }
-        LogicalType logicalType = f.schema().getLogicalType();
+        LogicalType logicalType = field.schema().getLogicalType();
         if (logicalType != null) {
           Conversion<?> conversion = getData().getConversionByClass(accessor.getField().getType(), logicalType);
           if (conversion != null) {
             try {
-              accessor.set(record,
-                  convert(readWithoutConversion(oldDatum, f.schema(), in), f.schema(), logicalType, conversion));
+              accessor.set(record, convert(readWithoutConversion(oldDatum, field.schema(), in), field.schema(),
+                  logicalType, conversion));
             } catch (IllegalAccessException e) {
-              throw new AvroRuntimeException("Failed to set " + f);
+              throw new AvroRuntimeException("Failed to set " + field);
             }
             return;
           }
         }
         try {
-          accessor.set(record, readWithoutConversion(oldDatum, f.schema(), in));
+          accessor.set(record, readWithoutConversion(oldDatum, field.schema(), in));
           return;
         } catch (IllegalAccessException e) {
-          throw new AvroRuntimeException("Failed to set " + f);
+          throw new AvroRuntimeException("Failed to set " + field);
         }
       }
     }
-    super.readField(record, f, oldDatum, in, state);
+    super.readField(record, field, oldDatum, in, state);
   }
 }
diff --git a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java
index ac8c319..d924c8e 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumReader.java
@@ -124,22 +124,22 @@ public class SpecificDatumReader<T> extends GenericDatumReader<T> {
   }
 
   @Override
-  protected void readField(Object r, Schema.Field f, Object oldDatum, ResolvingDecoder in, Object state)
+  protected void readField(Object record, Schema.Field field, Object oldDatum, ResolvingDecoder in, Object state)
       throws IOException {
-    if (r instanceof SpecificRecordBase) {
-      Conversion<?> conversion = ((SpecificRecordBase) r).getConversion(f.pos());
+    if (record instanceof SpecificRecordBase) {
+      Conversion<?> conversion = ((SpecificRecordBase) record).getConversion(field.pos());
 
       Object datum;
       if (conversion != null) {
-        datum = readWithConversion(oldDatum, f.schema(), f.schema().getLogicalType(), conversion, in);
+        datum = readWithConversion(oldDatum, field.schema(), field.schema().getLogicalType(), conversion, in);
       } else {
-        datum = readWithoutConversion(oldDatum, f.schema(), in);
+        datum = readWithoutConversion(oldDatum, field.schema(), in);
       }
 
-      getData().setField(r, f.name(), f.pos(), datum);
+      getData().setField(record, field.name(), field.pos(), datum);
 
     } else {
-      super.readField(r, f, oldDatum, in, state);
+      super.readField(record, field, oldDatum, in, state);
     }
   }
 }
diff --git a/lang/java/avro/src/main/java/org/apache/avro/util/internal/JacksonUtils.java b/lang/java/avro/src/main/java/org/apache/avro/util/internal/JacksonUtils.java
index fc4560e..7014f6c 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/util/internal/JacksonUtils.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/util/internal/JacksonUtils.java
@@ -123,22 +123,22 @@ public class JacksonUtils {
         return jsonNode.textValue().getBytes(StandardCharsets.ISO_8859_1);
       }
     } else if (jsonNode.isArray()) {
-      List l = new ArrayList();
+      List<Object> l = new ArrayList<>();
       for (JsonNode node : jsonNode) {
         l.add(toObject(node, schema == null ? null : schema.getElementType()));
       }
       return l;
     } else if (jsonNode.isObject()) {
-      Map m = new LinkedHashMap();
+      Map<Object, Object> m = new LinkedHashMap<>();
       for (Iterator<String> it = jsonNode.fieldNames(); it.hasNext();) {
         String key = it.next();
-        Schema s = null;
-        if (schema == null) {
-          s = null;
-        } else if (schema.getType().equals(Schema.Type.MAP)) {
+        final Schema s;
+        if (schema != null && schema.getType().equals(Schema.Type.MAP)) {
           s = schema.getValueType();
-        } else if (schema.getType().equals(Schema.Type.RECORD)) {
+        } else if (schema != null && schema.getType().equals(Schema.Type.RECORD)) {
           s = schema.getField(key).schema();
+        } else {
+          s = null;
         }
         Object value = toObject(jsonNode.get(key), s);
         m.put(key, value);
diff --git a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
index 25aefb3..b5f2bd0 100644
--- a/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
+++ b/lang/java/avro/src/test/java/org/apache/avro/TestSchemaCompatibility.java
@@ -20,11 +20,13 @@ package org.apache.avro;
 import static java.util.Arrays.asList;
 import static org.apache.avro.SchemaCompatibility.*;
 import static org.apache.avro.TestSchemas.*;
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
-import java.io.ByteArrayOutputStream;
+import java.io.*;
 import java.util.*;
+import java.util.stream.Collectors;
 
+import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericData.EnumSymbol;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericDatumWriter;
@@ -34,7 +36,9 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/** Unit-tests for SchemaCompatibility. */
+/**
+ * Unit-tests for SchemaCompatibility.
+ */
 public class TestSchemaCompatibility {
   private static final Logger LOG = LoggerFactory.getLogger(TestSchemaCompatibility.class);
   // -----------------------------------------------------------------------------------------------
@@ -43,7 +47,7 @@ public class TestSchemaCompatibility {
       new Schema.Field("oldfield1", INT_SCHEMA, null, null), new Schema.Field("oldfield2", STRING_SCHEMA, null, null)));
 
   @Test
-  public void testValidateSchemaPairMissingField() throws Exception {
+  public void testValidateSchemaPairMissingField() {
     final List<Schema.Field> readerFields = list(new Schema.Field("oldfield1", INT_SCHEMA, null, null));
     final Schema reader = Schema.createRecord(readerFields);
     final SchemaCompatibility.SchemaPairCompatibility expectedResult = new SchemaCompatibility.SchemaPairCompatibility(
@@ -55,7 +59,7 @@ public class TestSchemaCompatibility {
   }
 
   @Test
-  public void testValidateSchemaPairMissingSecondField() throws Exception {
+  public void testValidateSchemaPairMissingSecondField() {
     final List<Schema.Field> readerFields = list(new Schema.Field("oldfield2", STRING_SCHEMA, null, null));
     final Schema reader = Schema.createRecord(readerFields);
     final SchemaCompatibility.SchemaPairCompatibility expectedResult = new SchemaCompatibility.SchemaPairCompatibility(
@@ -67,7 +71,7 @@ public class TestSchemaCompatibility {
   }
 
   @Test
-  public void testValidateSchemaPairAllFields() throws Exception {
+  public void testValidateSchemaPairAllFields() {
     final List<Schema.Field> readerFields = list(new Schema.Field("oldfield1", INT_SCHEMA, null, null),
         new Schema.Field("oldfield2", STRING_SCHEMA, null, null));
     final Schema reader = Schema.createRecord(readerFields);
@@ -80,7 +84,7 @@ public class TestSchemaCompatibility {
   }
 
   @Test
-  public void testValidateSchemaNewFieldWithDefault() throws Exception {
+  public void testValidateSchemaNewFieldWithDefault() {
     final List<Schema.Field> readerFields = list(new Schema.Field("oldfield1", INT_SCHEMA, null, null),
         new Schema.Field("newfield1", INT_SCHEMA, null, 42));
     final Schema reader = Schema.createRecord(readerFields);
@@ -93,7 +97,7 @@ public class TestSchemaCompatibility {
   }
 
   @Test
-  public void testValidateSchemaNewField() throws Exception {
+  public void testValidateSchemaNewField() {
     final List<Schema.Field> readerFields = list(new Schema.Field("oldfield1", INT_SCHEMA, null, null),
         new Schema.Field("newfield1", INT_SCHEMA, null, null));
     final Schema reader = Schema.createRecord(readerFields);
@@ -112,7 +116,7 @@ public class TestSchemaCompatibility {
   }
 
   @Test
-  public void testValidateArrayWriterSchema() throws Exception {
+  public void testValidateArrayWriterSchema() {
     final Schema validReader = Schema.createArray(STRING_SCHEMA);
     final Schema invalidReader = Schema.createMap(STRING_SCHEMA);
     final SchemaCompatibility.SchemaPairCompatibility validResult = new SchemaCompatibility.SchemaPairCompatibility(
@@ -132,7 +136,7 @@ public class TestSchemaCompatibility {
   }
 
   @Test
-  public void testValidatePrimitiveWriterSchema() throws Exception {
+  public void testValidatePrimitiveWriterSchema() {
     final Schema validReader = Schema.create(Schema.Type.STRING);
     final SchemaCompatibility.SchemaPairCompatibility validResult = new SchemaCompatibility.SchemaPairCompatibility(
         SchemaCompatibility.SchemaCompatibilityResult.compatible(), validReader, STRING_SCHEMA,
@@ -149,7 +153,9 @@ public class TestSchemaCompatibility {
     assertEquals(invalidResult, checkReaderWriterCompatibility(INT_SCHEMA, STRING_SCHEMA));
   }
 
-  /** Reader union schema must contain all writer union branches. */
+  /**
+   * Reader union schema must contain all writer union branches.
+   */
   @Test
   public void testUnionReaderWriterSubsetIncompatibility() {
     final Schema unionWriter = Schema.createUnion(list(INT_SCHEMA, STRING_SCHEMA, LONG_SCHEMA));
@@ -160,7 +166,9 @@ public class TestSchemaCompatibility {
 
   // -----------------------------------------------------------------------------------------------
 
-  /** Collection of reader/writer schema pair that are compatible. */
+  /**
+   * Collection of reader/writer schema pair that are compatible.
+   */
   public static final List<ReaderWriter> COMPATIBLE_READER_WRITER_TEST_CASES = list(
       new ReaderWriter(BOOLEAN_SCHEMA, BOOLEAN_SCHEMA),
 
@@ -282,7 +290,9 @@ public class TestSchemaCompatibility {
 
   // -----------------------------------------------------------------------------------------------
 
-  /** Tests reader/writer compatibility validation. */
+  /**
+   * Tests reader/writer compatibility validation.
+   */
   @Test
   public void testReaderWriterCompatibility() {
     for (ReaderWriter readerWriter : COMPATIBLE_READER_WRITER_TEST_CASES) {
@@ -303,13 +313,19 @@ public class TestSchemaCompatibility {
    * value.
    */
   private static final class DecodingTestCase {
-    /** Writer schema used to encode the datum. */
+    /**
+     * Writer schema used to encode the datum.
+     */
     private final Schema mWriterSchema;
 
-    /** Datum to encode according to the specified writer schema. */
+    /**
+     * Datum to encode according to the specified writer schema.
+     */
     private final Object mDatum;
 
-    /** Reader schema used to decode the datum encoded using the writer schema. */
+    /**
+     * Reader schema used to decode the datum encoded using the writer schema.
+     */
     private final Schema mReaderSchema;
 
     /**
@@ -372,7 +388,9 @@ public class TestSchemaCompatibility {
 
       new DecodingTestCase(INT_STRING_UNION_SCHEMA, "the string", STRING_UNION_SCHEMA, new Utf8("the string")));
 
-  /** Tests the reader/writer compatibility at decoding time. */
+  /**
+   * Tests the reader/writer compatibility at decoding time.
+   */
   @Test
   public void testReaderWriterDecodingCompatibility() throws Exception {
     for (DecodingTestCase testCase : DECODING_COMPATIBILITY_TEST_CASES) {
@@ -404,10 +422,61 @@ public class TestSchemaCompatibility {
     }
   }
 
-  Deque<String> asDeqeue(String... args) {
-    List<String> x = Arrays.asList(args);
-    Collections.reverse(x);
-    Deque<String> dq = new ArrayDeque<>(x);
-    return dq;
+  private Schema readSchemaFromResources(String name) throws IOException {
+    try (InputStream inputStream = getClass().getClassLoader().getResourceAsStream(name)) {
+      final String result = new BufferedReader(new InputStreamReader(inputStream)).lines()
+          .collect(Collectors.joining("\n"));
+
+      return new Schema.Parser().parse(result);
+    }
+  }
+
+  @Test
+  public void checkResolvingDecoder() throws IOException {
+    final Schema locationSchema = readSchemaFromResources("schema-location.json");
+    final Schema writeSchema = readSchemaFromResources("schema-location-write.json");
+
+    // For the read schema the long field has been removed
+    // And a new field has been added, called long_r2
+    // This one should be null.
+    final Schema readSchema = readSchemaFromResources("schema-location-read.json");
+
+    // Create some testdata
+    GenericData.Record record = new GenericData.Record(writeSchema);
+    GenericData.Record location = new GenericData.Record(locationSchema);
+
+    location.put("lat", 52.995143f);
+    location.put("long", -1.539054f);
+
+    HashMap<String, GenericData.Record> locations = new HashMap<>();
+    locations.put("l1", location);
+    record.put("location", locations);
+
+    // Write the record to bytes
+    byte[] payload;
+    try (ByteArrayOutputStream bbos = new ByteArrayOutputStream()) {
+      DatumWriter<GenericData.Record> datumWriter = new GenericDatumWriter<>(writeSchema);
+      Encoder enc = EncoderFactory.get().binaryEncoder(bbos, null);
+      datumWriter.write(record, enc);
+      enc.flush();
+
+      payload = bbos.toByteArray();
+    }
+
+    // Read the record, and decode it using the read with the long
+    // And project it using the other schema with the long_r2
+    BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(payload, null);
+    GenericDatumReader<GenericData.Record> reader = new GenericDatumReader<>();
+    reader.setSchema(writeSchema);
+    reader.setExpected(readSchema);
+
+    // Get the object we're looking for
+    GenericData.Record r = reader.read(null, decoder);
+    HashMap<Utf8, GenericData.Record> locs = (HashMap<Utf8, GenericData.Record>) r.get("location");
+    GenericData.Record loc = locs.get(new Utf8("l1"));
+
+    assertNotNull(loc.get("lat"));
+    // This is a new field, and should be null
+    assertNull(loc.get("long_r2"));
   }
 }
diff --git a/lang/java/avro/src/test/resources/schema-location-read.json b/lang/java/avro/src/test/resources/schema-location-read.json
new file mode 100644
index 0000000..c570ba4
--- /dev/null
+++ b/lang/java/avro/src/test/resources/schema-location-read.json
@@ -0,0 +1,28 @@
+{
+  "type": "record",
+  "name": "table",
+  "fields": [{
+    "name": "location",
+    "type": ["null", {
+      "type": "map",
+      "values": ["null", {
+        "type": "record",
+        "name": "r7",
+        "fields": [{
+          "name": "lat",
+          "type": "float",
+          "field-id": 1
+        }, {
+          "name": "long_r2",
+          "type": ["null", "float"],
+          "default": null,
+          "field-id": 2
+        }]
+      }],
+      "key-id": 6,
+      "value-id": 7
+    }],
+    "default": null,
+    "field-id": 5
+  }]
+}
diff --git a/lang/java/avro/src/test/resources/schema-location-write.json b/lang/java/avro/src/test/resources/schema-location-write.json
new file mode 100644
index 0000000..5ac7325
--- /dev/null
+++ b/lang/java/avro/src/test/resources/schema-location-write.json
@@ -0,0 +1,28 @@
+{
+  "type": "record",
+  "name": "table",
+  "fields": [{
+    "name": "location",
+    "type": ["null", {
+      "type": "map",
+      "values": ["null", {
+        "type": "record",
+        "name": "r7",
+        "fields": [{
+          "name": "lat",
+          "type": "float",
+          "field-id": 1
+        }, {
+          "name": "long",
+          "type": ["null", "float"],
+          "default": null,
+          "field-id": 2
+        }]
+      }],
+      "key-id": 6,
+      "value-id": 7
+    }],
+    "default": null,
+    "field-id": 5
+  }]
+}
diff --git a/lang/java/avro/src/test/resources/schema-location.json b/lang/java/avro/src/test/resources/schema-location.json
new file mode 100644
index 0000000..0baedad
--- /dev/null
+++ b/lang/java/avro/src/test/resources/schema-location.json
@@ -0,0 +1,14 @@
+{
+  "type": "record",
+  "name": "r7",
+  "fields": [{
+    "name": "lat",
+    "type": "float",
+    "field-id": 1
+  }, {
+    "name": "long",
+    "type": ["null", "float"],
+    "default": null,
+    "field-id": 2
+  }]
+}
diff --git a/lang/java/protobuf/src/main/java/org/apache/avro/protobuf/ProtobufData.java b/lang/java/protobuf/src/main/java/org/apache/avro/protobuf/ProtobufData.java
index 5c403bb..a7c847c 100644
--- a/lang/java/protobuf/src/main/java/org/apache/avro/protobuf/ProtobufData.java
+++ b/lang/java/protobuf/src/main/java/org/apache/avro/protobuf/ProtobufData.java
@@ -78,8 +78,8 @@ public class ProtobufData extends GenericData {
   }
 
   @Override
-  public void setField(Object r, String n, int pos, Object o) {
-    setField(r, n, pos, o, getRecordState(r, getSchema(r.getClass())));
+  public void setField(Object r, String n, int pos, Object value) {
+    setField(r, n, pos, value, getRecordState(r, getSchema(r.getClass())));
   }
 
   @Override
@@ -88,17 +88,17 @@ public class ProtobufData extends GenericData {
   }
 
   @Override
-  protected void setField(Object r, String n, int pos, Object o, Object state) {
-    Builder b = (Builder) r;
-    FieldDescriptor f = ((FieldDescriptor[]) state)[pos];
+  protected void setField(Object record, String name, int position, Object value, Object state) {
+    Builder b = (Builder) record;
+    FieldDescriptor f = ((FieldDescriptor[]) state)[position];
     switch (f.getType()) {
     case MESSAGE:
-      if (o == null) {
+      if (value == null) {
         b.clearField(f);
         break;
       }
     default:
-      b.setField(f, o);
+      b.setField(f, value);
     }
   }
 
diff --git a/lang/java/thrift/src/main/java/org/apache/avro/thrift/ThriftData.java b/lang/java/thrift/src/main/java/org/apache/avro/thrift/ThriftData.java
index a603b0b..eaf11c7 100644
--- a/lang/java/thrift/src/main/java/org/apache/avro/thrift/ThriftData.java
+++ b/lang/java/thrift/src/main/java/org/apache/avro/thrift/ThriftData.java
@@ -73,8 +73,8 @@ public class ThriftData extends GenericData {
   }
 
   @Override
-  public void setField(Object r, String n, int pos, Object o) {
-    setField(r, n, pos, o, getRecordState(r, getSchema(r.getClass())));
+  public void setField(Object r, String n, int pos, Object value) {
+    setField(r, n, pos, value, getRecordState(r, getSchema(r.getClass())));
   }
 
   @Override
@@ -83,10 +83,10 @@ public class ThriftData extends GenericData {
   }
 
   @Override
-  protected void setField(Object r, String n, int pos, Object v, Object state) {
-    if (v == null && r instanceof TUnion)
+  protected void setField(Object record, String name, int position, Object value, Object state) {
+    if (value == null && record instanceof TUnion)
       return;
-    ((TBase) r).setFieldValue(((TFieldIdEnum[]) state)[pos], v);
+    ((TBase) record).setFieldValue(((TFieldIdEnum[]) state)[position], value);
   }
 
   @Override


Mime
View raw message