You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by rs...@apache.org on 2022/02/18 07:16:37 UTC

[avro] branch branch-1.11 updated: AVRO-3375: Better error messages out of generic and specific DatumWriters (#1526)

This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-1.11 by this push:
     new b0fbd92  AVRO-3375: Better error messages out of generic and specific DatumWriters (#1526)
b0fbd92 is described below

commit b0fbd92ddce1f44bd7b8b41f14069ec2e360087b
Author: Radai Rosenblatt <ra...@gmail.com>
AuthorDate: Thu Feb 17 23:09:57 2022 -0800

    AVRO-3375: Better error messages out of generic and specific DatumWriters (#1526)
    
    * AVRO-3375 - better error messages out of generic and specific DatumWriters
    
    this commit introduces "path tracking" to trace the path from a root datum object to be written to the location of 3 common serialization errors:
    NullPointerExceptions, ClassCastExceptions and AvroTypeExceptions.
    
    path tracing is done using AvroPath-like path components and predicates that would maybe become part of a future implementation of AVRO-992
    
    note that path tracing does not woth if custom coders are in use for serialization (neither does the tracking code this commit replaces).
    
    * address review comments
    
    * address more review comments
    
    Co-authored-by: Radai Rosenblatt <rr...@rrosenbl-mn2.linkedin.biz>
---
 .../apache/avro/generic/GenericDatumWriter.java    |  80 ++++++++---
 .../apache/avro/path/ArrayPositionPredicate.java   |  35 +++++
 .../java/org/apache/avro/path/LocationStep.java    |  47 +++++++
 .../java/org/apache/avro/path/MapKeyPredicate.java |  43 ++++++
 .../java/org/apache/avro/path/PathElement.java     |  25 ++++
 .../java/org/apache/avro/path/PathPredicate.java   |  26 ++++
 .../org/apache/avro/path/PathTracingException.java |  46 +++++++
 .../apache/avro/path/PositionalPathPredicate.java  |  25 ++++
 .../apache/avro/path/TracingAvroTypeException.java |  66 +++++++++
 .../avro/path/TracingClassCastException.java       |  83 ++++++++++++
 .../avro/path/TracingNullPointException.java       |  92 +++++++++++++
 .../org/apache/avro/path/UnionTypePredicate.java   |  35 +++++
 .../java/org/apache/avro/path/package-info.java    |  25 ++++
 .../main/java/org/apache/avro/path/package.html    |  29 ++++
 .../org/apache/avro/specific/SpecificData.java     |   5 +-
 .../apache/avro/specific/SpecificDatumWriter.java  |  17 ++-
 .../main/java/org/apache/avro/util/SchemaUtil.java |  56 ++++++++
 .../avro/generic/TestGenericDatumWriter.java       | 147 ++++++++++++++++++++-
 .../avro/specific/TestSpecificDatumWriter.java     | 132 +++++++++++++++++-
 share/test/schemas/RecordWithRequiredFields.avsc   |  53 ++++++++
 20 files changed, 1035 insertions(+), 32 deletions(-)

diff --git a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java
index a692c80..deeac0b 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/generic/GenericDatumWriter.java
@@ -32,9 +32,17 @@ import org.apache.avro.Conversions;
 import org.apache.avro.LogicalType;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
+import org.apache.avro.path.TracingAvroTypeException;
 import org.apache.avro.UnresolvedUnionException;
 import org.apache.avro.io.DatumWriter;
 import org.apache.avro.io.Encoder;
+import org.apache.avro.path.ArrayPositionPredicate;
+import org.apache.avro.path.LocationStep;
+import org.apache.avro.path.MapKeyPredicate;
+import org.apache.avro.path.TracingClassCastException;
+import org.apache.avro.path.TracingNullPointException;
+import org.apache.avro.path.UnionTypePredicate;
+import org.apache.avro.util.SchemaUtil;
 
 /** {@link DatumWriter} for generic Java objects. */
 public class GenericDatumWriter<D> implements DatumWriter<D> {
@@ -70,7 +78,11 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
 
   public void write(D datum, Encoder out) throws IOException {
     Objects.requireNonNull(out, "Encoder cannot be null");
-    write(root, datum, out);
+    try {
+      write(root, datum, out);
+    } catch (TracingNullPointException | TracingClassCastException | TracingAvroTypeException e) {
+      throw e.summarize(root);
+    }
   }
 
   /** Called to write data. */
@@ -125,8 +137,10 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
 
   /** Called to write data. */
   protected void writeWithoutConversion(Schema schema, Object datum, Encoder out) throws IOException {
+    int unionIndex = -1;
+    Schema.Type schemaType = schema.getType();
     try {
-      switch (schema.getType()) {
+      switch (schemaType) {
       case RECORD:
         writeRecord(schema, datum, out);
         break;
@@ -140,9 +154,9 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
         writeMap(schema, datum, out);
         break;
       case UNION:
-        int index = resolveUnion(schema, datum);
-        out.writeIndex(index);
-        write(schema.getTypes().get(index), datum, out);
+        unionIndex = resolveUnion(schema, datum);
+        out.writeIndex(unionIndex);
+        write(schema.getTypes().get(unionIndex), datum, out);
         break;
       case FIXED:
         writeFixed(schema, datum, out);
@@ -174,8 +188,18 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
       default:
         error(schema, datum);
       }
+    } catch (TracingNullPointException | TracingClassCastException | TracingAvroTypeException e) {
+      if (schemaType == Schema.Type.UNION) {
+        e.tracePath(new UnionTypePredicate(schema.getTypes().get(unionIndex).getName()));
+      }
+      // writeArray() and writeMap() have their own handling
+      throw e;
     } catch (NullPointerException e) {
-      throw npe(e, " of " + schema.getFullName());
+      throw new TracingNullPointException(e, schema, false);
+    } catch (ClassCastException e) {
+      throw new TracingClassCastException(e, datum, schema, false);
+    } catch (AvroTypeException e) {
+      throw new TracingAvroTypeException(e);
     }
   }
 
@@ -223,6 +247,9 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
       final UnresolvedUnionException unresolvedUnionException = new UnresolvedUnionException(f.schema(), f, value);
       unresolvedUnionException.addSuppressed(uue);
       throw unresolvedUnionException;
+    } catch (TracingNullPointException | TracingClassCastException | TracingAvroTypeException e) {
+      e.tracePath(new LocationStep(".", f.name()));
+      throw e;
     } catch (NullPointerException e) {
       throw npe(e, " in field " + f.name());
     } catch (ClassCastException cce) {
@@ -237,8 +264,11 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
    * representations.
    */
   protected void writeEnum(Schema schema, Object datum, Encoder out) throws IOException {
-    if (!data.isEnum(datum))
-      throw new AvroTypeException("Not an enum: " + datum + " for schema: " + schema);
+    if (!data.isEnum(datum)) {
+      AvroTypeException cause = new AvroTypeException(
+          "value " + SchemaUtil.describe(datum) + " is not a " + SchemaUtil.describe(schema));
+      throw new TracingAvroTypeException(cause);
+    }
     out.writeEnum(schema.getEnumOrdinal(datum.toString()));
   }
 
@@ -254,7 +284,12 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
     out.setItemCount(size);
     for (Iterator<? extends Object> it = getArrayElements(datum); it.hasNext();) {
       out.startItem();
-      write(element, it.next(), out);
+      try {
+        write(element, it.next(), out);
+      } catch (TracingNullPointException | TracingClassCastException | TracingAvroTypeException e) {
+        e.tracePath(new ArrayPositionPredicate(actualSize));
+        throw e;
+      }
       actualSize++;
     }
     out.writeArrayEnd();
@@ -276,18 +311,16 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
    * Called by the default implementation of {@link #writeArray} to get the size
    * of an array. The default implementation is for {@link Collection}.
    */
-  @SuppressWarnings("unchecked")
   protected long getArraySize(Object array) {
-    return ((Collection) array).size();
+    return ((Collection<?>) array).size();
   }
 
   /**
    * Called by the default implementation of {@link #writeArray} to enumerate
    * array elements. The default implementation is for {@link Collection}.
    */
-  @SuppressWarnings("unchecked")
-  protected Iterator<? extends Object> getArrayElements(Object array) {
-    return ((Collection) array).iterator();
+  protected Iterator<?> getArrayElements(Object array) {
+    return ((Collection<?>) array).iterator();
   }
 
   /**
@@ -301,8 +334,21 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
     out.setItemCount(size);
     for (Map.Entry<Object, Object> entry : getMapEntries(datum)) {
       out.startItem();
-      writeString(entry.getKey().toString(), out);
-      write(value, entry.getValue(), out);
+      String key;
+      try {
+        key = entry.getKey().toString();
+      } catch (NullPointerException npe) {
+        TracingNullPointException tnpe = new TracingNullPointException(npe, Schema.create(Schema.Type.STRING), false);
+        tnpe.tracePath(new MapKeyPredicate(null));
+        throw tnpe;
+      }
+      writeString(key, out);
+      try {
+        write(value, entry.getValue(), out);
+      } catch (TracingNullPointException | TracingClassCastException | TracingAvroTypeException e) {
+        e.tracePath(new MapKeyPredicate(key));
+        throw e;
+      }
       actualSize++;
     }
     out.writeMapEnd();
@@ -363,7 +409,7 @@ public class GenericDatumWriter<D> implements DatumWriter<D> {
   }
 
   private void error(Schema schema, Object datum) {
-    throw new AvroTypeException("Not a " + schema + ": " + datum);
+    throw new AvroTypeException("value " + SchemaUtil.describe(datum) + " is not a " + SchemaUtil.describe(schema));
   }
 
 }
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/ArrayPositionPredicate.java b/lang/java/avro/src/main/java/org/apache/avro/path/ArrayPositionPredicate.java
new file mode 100644
index 0000000..480d026
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/ArrayPositionPredicate.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * Returns items by their position (numeric index) in an array
+ */
+public class ArrayPositionPredicate implements PositionalPathPredicate {
+  private final long index;
+
+  public ArrayPositionPredicate(long index) {
+    this.index = index;
+  }
+
+  @Override
+  public String toString() {
+    return "[" + index + "]";
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/LocationStep.java b/lang/java/avro/src/main/java/org/apache/avro/path/LocationStep.java
new file mode 100644
index 0000000..c8442df
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/LocationStep.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * Selects items based on their "path" (name of a property under which they are
+ * stored) relative to the context.
+ */
+public class LocationStep implements PathElement {
+  /**
+   * selector part of location step. either "." or ".."
+   */
+  private final String selector;
+  /**
+   * name of a property to select
+   */
+  private final String propertyName;
+
+  public LocationStep(String selector, String propertyName) {
+    this.selector = selector;
+    this.propertyName = propertyName;
+  }
+
+  @Override
+  public String toString() {
+    if (propertyName == null || propertyName.isEmpty()) {
+      return selector;
+    }
+    return selector + propertyName;
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/MapKeyPredicate.java b/lang/java/avro/src/main/java/org/apache/avro/path/MapKeyPredicate.java
new file mode 100644
index 0000000..b183d84
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/MapKeyPredicate.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * Returns items by their position (string key under which they are stored) in a
+ * map
+ */
+public class MapKeyPredicate implements PositionalPathPredicate {
+  private final String key;
+
+  public MapKeyPredicate(String key) {
+    this.key = key;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  @Override
+  public String toString() {
+    if (key == null) {
+      return "";
+    }
+    return "[\"" + key + "\"]";
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/PathElement.java b/lang/java/avro/src/main/java/org/apache/avro/path/PathElement.java
new file mode 100644
index 0000000..f3be4dc
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/PathElement.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * root interface for all pieces of an AvroPath expression
+ */
+public interface PathElement {
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/PathPredicate.java b/lang/java/avro/src/main/java/org/apache/avro/path/PathPredicate.java
new file mode 100644
index 0000000..0928946
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/PathPredicate.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * a predicate is a filter that restricts items selected by a
+ * {@link LocationStep}
+ */
+public interface PathPredicate extends PathElement {
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/PathTracingException.java b/lang/java/avro/src/main/java/org/apache/avro/path/PathTracingException.java
new file mode 100644
index 0000000..ac9ba51
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/PathTracingException.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+import org.apache.avro.Schema;
+
+/**
+ * interface for exceptions that can trace the AvroPath of an error
+ *
+ * @param <T> the regular (user-facing) exception that will be
+ *            {@link #summarize(Schema)}ed out of this class
+ */
+public interface PathTracingException<T extends Throwable> {
+  /**
+   * appends a path element to the trace. expected to be called in reverse-order
+   * as the exception bubbles up the stack
+   *
+   * @param step an AvroPath step tracing back from the location of the original
+   *             exception towards the root of the data graph
+   */
+  void tracePath(PathElement step);
+
+  /**
+   * produces a user-facing exception to be thrown back out to user code
+   * 
+   * @param root the root object for the operation that generated the exception
+   * @return an exception
+   */
+  T summarize(Schema root);
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/PositionalPathPredicate.java b/lang/java/avro/src/main/java/org/apache/avro/path/PositionalPathPredicate.java
new file mode 100644
index 0000000..3c9751e
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/PositionalPathPredicate.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * filters items by their context position
+ */
+public interface PositionalPathPredicate extends PathPredicate {
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/TracingAvroTypeException.java b/lang/java/avro/src/main/java/org/apache/avro/path/TracingAvroTypeException.java
new file mode 100644
index 0000000..4aed18b
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/TracingAvroTypeException.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.avro.AvroTypeException;
+import org.apache.avro.Schema;
+import org.apache.avro.util.SchemaUtil;
+
+/**
+ * an {@link AvroTypeException} with extra fields used to trace back the path to
+ * a bad value through an object graph
+ */
+public class TracingAvroTypeException extends AvroTypeException implements PathTracingException<AvroTypeException> {
+  private final List<PathElement> reversePath;
+
+  public TracingAvroTypeException(AvroTypeException cause) {
+    super(cause.getMessage(), cause);
+    this.reversePath = new ArrayList<>(3); // expected to be short
+  }
+
+  @Override
+  public void tracePath(PathElement step) {
+    reversePath.add(step);
+  }
+
+  @Override
+  public AvroTypeException summarize(Schema root) {
+    AvroTypeException cause = (AvroTypeException) getCause();
+
+    StringBuilder sb = new StringBuilder();
+    sb.append(cause.getMessage());
+
+    if (reversePath != null && !reversePath.isEmpty()) {
+      sb.append(" at ");
+      if (root != null) {
+        sb.append(SchemaUtil.describe(root));
+      }
+      for (int i = reversePath.size() - 1; i >= 0; i--) {
+        PathElement step = reversePath.get(i);
+        sb.append(step.toString());
+      }
+    }
+    AvroTypeException summary = new AvroTypeException(sb.toString());
+    summary.initCause(cause);
+    return summary;
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/TracingClassCastException.java b/lang/java/avro/src/main/java/org/apache/avro/path/TracingClassCastException.java
new file mode 100644
index 0000000..87607ae
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/TracingClassCastException.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.path;
+
+import org.apache.avro.Schema;
+import org.apache.avro.util.SchemaUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * a {@link ClassCastException} with extra fields used to trace back the path to
+ * a bad value through an object graph
+ */
+public class TracingClassCastException extends ClassCastException implements PathTracingException<ClassCastException> {
+  private final ClassCastException cause;
+  private final Object datum;
+  private final Schema expected;
+  private final boolean customCoderUsed;
+  private final List<PathElement> reversePath;
+
+  public TracingClassCastException(ClassCastException cause, Object datum, Schema expected, boolean customCoderUsed) {
+    this.cause = cause;
+    this.datum = datum;
+    this.expected = expected;
+    this.customCoderUsed = customCoderUsed;
+    this.reversePath = new ArrayList<>(3); // assume short
+  }
+
+  @Override
+  public void tracePath(PathElement step) {
+    reversePath.add(step);
+  }
+
+  @Override
+  public synchronized ClassCastException getCause() {
+    return cause;
+  }
+
+  /**
+   * @return a hopefully helpful error message
+   */
+  @Override
+  public ClassCastException summarize(Schema root) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("value ").append(SchemaUtil.describe(datum));
+    sb.append(" cannot be cast to expected type ").append(SchemaUtil.describe(expected));
+    if (reversePath == null || reversePath.isEmpty()) {
+      // very simple "shallow" NPE, no nesting at all, or custom coders used means we
+      // have no data
+      if (customCoderUsed) {
+        sb.append(". No further details available as custom coders were used");
+      }
+    } else {
+      sb.append(" at ");
+      if (root != null) {
+        sb.append(SchemaUtil.describe(root));
+      }
+      for (int i = reversePath.size() - 1; i >= 0; i--) {
+        PathElement step = reversePath.get(i);
+        sb.append(step.toString());
+      }
+    }
+    ClassCastException summary = new ClassCastException(sb.toString());
+    summary.initCause(cause);
+    return summary;
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/TracingNullPointException.java b/lang/java/avro/src/main/java/org/apache/avro/path/TracingNullPointException.java
new file mode 100644
index 0000000..fabfc76
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/TracingNullPointException.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.path;
+
+import org.apache.avro.Schema;
+import org.apache.avro.util.SchemaUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * a {@link NullPointerException} with extra fields used to trace back the path
+ * to a null value through an object graph
+ */
+public class TracingNullPointException extends NullPointerException
+    implements PathTracingException<NullPointerException> {
+  private final NullPointerException cause;
+  private final Schema expected;
+  private final boolean customCoderUsed;
+  private final List<PathElement> reversePath;
+
+  public TracingNullPointException(NullPointerException cause, Schema expected, boolean customCoderUsed) {
+    this.cause = cause;
+    this.expected = expected;
+    this.customCoderUsed = customCoderUsed;
+    this.reversePath = new ArrayList<>(3); // assume short
+  }
+
+  @Override
+  public void tracePath(PathElement step) {
+    reversePath.add(step);
+  }
+
+  @Override
+  public synchronized NullPointerException getCause() {
+    return cause;
+  }
+
+  /**
+   * @return a hopefully helpful error message
+   */
+  @Override
+  public NullPointerException summarize(Schema root) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("null value for (non-nullable) ");
+    if (reversePath == null || reversePath.isEmpty()) {
+      // very simple "shallow" NPE, no nesting at all, or custom coders used means we
+      // have no data
+      if (customCoderUsed) {
+        sb.append("field or map key. No further details available as custom coders were used");
+      } else {
+        sb.append(SchemaUtil.describe(expected));
+      }
+    } else {
+      PathElement innerMostElement = reversePath.get(0);
+      boolean isNullMapKey = innerMostElement instanceof MapKeyPredicate
+          && ((MapKeyPredicate) innerMostElement).getKey() == null;
+      if (isNullMapKey) {
+        sb.delete(0, sb.length()); // clear
+        sb.append("null key in map");
+      } else {
+        sb.append(SchemaUtil.describe(expected));
+      }
+      sb.append(" at ");
+      if (root != null) {
+        sb.append(SchemaUtil.describe(root));
+      }
+      for (int i = reversePath.size() - 1; i >= 0; i--) {
+        PathElement step = reversePath.get(i);
+        sb.append(step.toString());
+      }
+    }
+    NullPointerException summary = new NullPointerException(sb.toString());
+    summary.initCause(cause);
+    return summary;
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/UnionTypePredicate.java b/lang/java/avro/src/main/java/org/apache/avro/path/UnionTypePredicate.java
new file mode 100644
index 0000000..01e30e1
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/UnionTypePredicate.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.avro.path;
+
+/**
+ * Returns items by their position (numeric index of type) in a union schema
+ */
+public class UnionTypePredicate implements PositionalPathPredicate {
+  private final String type;
+
+  public UnionTypePredicate(String type) {
+    this.type = type;
+  }
+
+  @Override
+  public String toString() {
+    return "[" + type + "]";
+  }
+}
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/package-info.java b/lang/java/avro/src/main/java/org/apache/avro/path/package-info.java
new file mode 100644
index 0000000..8b1dea4
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Interfaces and base classes for AvroPath. This functionality is
+ * <b>experimental</b>, meaning these APIs are not expected to be stable any
+ * time soon so use at your own risk. Feedback, however, would be very
+ * appreciated :-)
+ */
+package org.apache.avro.path;
diff --git a/lang/java/avro/src/main/java/org/apache/avro/path/package.html b/lang/java/avro/src/main/java/org/apache/avro/path/package.html
new file mode 100644
index 0000000..73ab0a7
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/path/package.html
@@ -0,0 +1,29 @@
+<html>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       https://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<body>
+  Interfaces and base classes for AvroPath.
+
+<p>
+  This functionality is <b>experimental</b>, meaning these APIs are not
+  expected to be stable any time soon so use at your own risk. Feedback,
+  however, would be very appreciated :-)
+</p>
+</body>
+</html>
diff --git a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
index f392868..46731b9 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java
@@ -31,6 +31,7 @@ import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.util.ClassUtils;
 import org.apache.avro.util.MapUtil;
+import org.apache.avro.util.SchemaUtil;
 import org.apache.avro.util.internal.ClassValueCache;
 
 import java.io.ObjectInput;
@@ -380,8 +381,8 @@ public class SpecificData extends GenericData {
       } else if (Map.class.isAssignableFrom(raw)) { // map
         java.lang.reflect.Type key = params[0];
         java.lang.reflect.Type value = params[1];
-        if (!(key instanceof Class && CharSequence.class.isAssignableFrom((Class) key)))
-          throw new AvroTypeException("Map key class not CharSequence: " + key);
+        if (!(key instanceof Class && CharSequence.class.isAssignableFrom((Class<?>) key)))
+          throw new AvroTypeException("Map key class not CharSequence: " + SchemaUtil.describe(key));
         return Schema.createMap(createSchema(value, names));
       } else {
         return createSchema(raw, names);
diff --git a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumWriter.java b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumWriter.java
index 4611847..1721403 100644
--- a/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumWriter.java
+++ b/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificDatumWriter.java
@@ -23,8 +23,12 @@ import org.apache.avro.AvroTypeException;
 import org.apache.avro.Conversion;
 import org.apache.avro.LogicalType;
 import org.apache.avro.Schema;
+import org.apache.avro.path.TracingAvroTypeException;
 import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.path.TracingClassCastException;
+import org.apache.avro.path.TracingNullPointException;
 import org.apache.avro.io.Encoder;
+import org.apache.avro.path.LocationStep;
 
 /**
  * {@link org.apache.avro.io.DatumWriter DatumWriter} for generated Java
@@ -77,7 +81,11 @@ public class SpecificDatumWriter<T> extends GenericDatumWriter<T> {
     if (datum instanceof SpecificRecordBase && this.getSpecificData().useCustomCoders()) {
       SpecificRecordBase d = (SpecificRecordBase) datum;
       if (d.hasCustomCoders()) {
-        d.customEncode(out);
+        try {
+          d.customEncode(out);
+        } catch (NullPointerException e) {
+          throw new TracingNullPointException(e, null, true);
+        }
         return;
       }
     }
@@ -98,10 +106,9 @@ public class SpecificDatumWriter<T> extends GenericDatumWriter<T> {
 
       try {
         writeWithoutConversion(fieldSchema, value, out);
-      } catch (NullPointerException e) {
-        throw npe(e, " in field '" + f.name() + "'");
-      } catch (ClassCastException cce) {
-        throw addClassCastMsg(cce, " in field '" + f.name() + "'");
+      } catch (TracingNullPointException | TracingClassCastException | TracingAvroTypeException e) {
+        e.tracePath(new LocationStep(".", f.name()));
+        throw e;
       } catch (AvroTypeException ate) {
         throw addAvroTypeMsg(ate, " in field '" + f.name() + "'");
       }
diff --git a/lang/java/avro/src/main/java/org/apache/avro/util/SchemaUtil.java b/lang/java/avro/src/main/java/org/apache/avro/util/SchemaUtil.java
new file mode 100644
index 0000000..0661981
--- /dev/null
+++ b/lang/java/avro/src/main/java/org/apache/avro/util/SchemaUtil.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.avro.util;
+
+import org.apache.avro.Schema;
+
+import java.util.StringJoiner;
+
+public class SchemaUtil {
+
+  private SchemaUtil() {
+    // utility class
+  }
+
+  public static String describe(Schema schema) {
+    if (schema == null) {
+      return "unknown";
+    }
+    switch (schema.getType()) {
+    case UNION:
+      StringJoiner csv = new StringJoiner(", ");
+      for (Schema branch : schema.getTypes()) {
+        csv.add(describe(branch));
+      }
+      return "[" + csv + "]";
+    case MAP:
+      return "Map<String, " + describe(schema.getValueType()) + ">";
+    case ARRAY:
+      return "List<" + describe(schema.getElementType()) + ">";
+    default:
+      return schema.getName();
+    }
+  }
+
+  public static String describe(Object datum) {
+    if (datum == null) {
+      return "null";
+    }
+    return datum + " (a " + datum.getClass().getName() + ")";
+  }
+}
diff --git a/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java b/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java
index 2d5bf20..6cc3673 100644
--- a/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java
+++ b/lang/java/avro/src/test/java/org/apache/avro/generic/TestGenericDatumWriter.java
@@ -23,8 +23,10 @@ import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.ConcurrentModificationException;
 import java.util.HashMap;
@@ -43,6 +45,7 @@ import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.Encoder;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.util.Utf8;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestGenericDatumWriter {
@@ -282,7 +285,7 @@ public class TestGenericDatumWriter {
     public void writeIndex(int unionIndex) throws IOException {
       e.writeIndex(unionIndex);
     }
-  };
+  }
 
   @Test(expected = AvroTypeException.class)
   public void writeDoesNotAllowStringForGenericEnum() throws IOException {
@@ -302,7 +305,7 @@ public class TestGenericDatumWriter {
 
   private enum AnEnum {
     ONE, TWO, THREE
-  };
+  }
 
   @Test(expected = AvroTypeException.class)
   public void writeDoesNotAllowJavaEnumForGenericEnum() throws IOException {
@@ -324,14 +327,139 @@ public class TestGenericDatumWriter {
   public void writeFieldWithDefaultWithExplicitNullDefaultInSchema() throws Exception {
     Schema schema = schemaWithExplicitNullDefault();
     GenericRecord record = createRecordWithDefaultField(schema);
-    writeObject(schema, record);
+    writeObject(record);
   }
 
   @Test
   public void writeFieldWithDefaultWithoutExplicitNullDefaultInSchema() throws Exception {
     Schema schema = schemaWithoutExplicitNullDefault();
     GenericRecord record = createRecordWithDefaultField(schema);
-    writeObject(schema, record);
+    writeObject(record);
+  }
+
+  @Test
+  public void testNestedNPEErrorClarity() throws Exception {
+    GenericData.Record topLevelRecord = buildComplexRecord();
+    @SuppressWarnings("unchecked")
+    Map<String, GenericData.Record> map = (Map<String, GenericData.Record>) ((List<GenericData.Record>) ((GenericData.Record) topLevelRecord
+        .get("unionField")).get("arrayField")).get(0).get("mapField");
+    map.get("a").put("strField", null);
+    try {
+      writeObject(topLevelRecord);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField[\"a\"].strField"));
+    }
+  }
+
+  @Test
+  public void testNPEForMapKeyErrorClarity() throws Exception {
+    GenericData.Record topLevelRecord = buildComplexRecord();
+    @SuppressWarnings("unchecked")
+    Map<String, GenericData.Record> map = (Map<String, GenericData.Record>) ((List<GenericData.Record>) ((GenericData.Record) topLevelRecord
+        .get("unionField")).get("arrayField")).get(0).get("mapField");
+    map.put(null, map.get("a")); // value is valid, but key is null
+    try {
+      writeObject(topLevelRecord);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("null key in map at RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField"));
+    }
+  }
+
+  @Test
+  public void testShortPathNPEErrorClarity() throws Exception {
+    try {
+      writeObject(Schema.create(Schema.Type.STRING), null);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(),
+          expected.getMessage().contains("null value for (non-nullable) string"));
+    }
+  }
+
+  @Test
+  public void testNestedCCEErrorClarity() throws Exception {
+    GenericData.Record topLevelRecord = buildComplexRecord();
+    @SuppressWarnings("unchecked")
+    Map<String, GenericData.Record> map = (Map<String, GenericData.Record>) ((List<GenericData.Record>) ((GenericData.Record) topLevelRecord
+        .get("unionField")).get("arrayField")).get(0).get("mapField");
+    map.get("a").put("strField", 42); // not a string
+    try {
+      writeObject(topLevelRecord);
+      Assert.fail("expected to throw");
+    } catch (ClassCastException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField[\"a\"].strField"));
+    }
+  }
+
+  @Test
+  public void testShortPathCCEErrorClarity() throws Exception {
+    try {
+      writeObject(Schema.create(Schema.Type.STRING), 42);
+      Assert.fail("expected to throw");
+    } catch (ClassCastException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(),
+          expected.getMessage().contains("value 42 (a java.lang.Integer) cannot be cast to expected type string"));
+    }
+  }
+
+  @Test
+  public void testNestedATEErrorClarity() throws Exception {
+    GenericData.Record topLevelRecord = buildComplexRecord();
+    @SuppressWarnings("unchecked")
+    Map<String, GenericData.Record> map = (Map<String, GenericData.Record>) ((List<GenericData.Record>) ((GenericData.Record) topLevelRecord
+        .get("unionField")).get("arrayField")).get(0).get("mapField");
+    map.get("a").put("enumField", 42); // not an enum
+    try {
+      writeObject(topLevelRecord);
+      Assert.fail("expected to throw");
+    } catch (AvroTypeException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField[\"a\"].enumField"));
+      Assert.assertTrue("unexpected message " + expected.getMessage(),
+          expected.getMessage().contains("42 (a java.lang.Integer) is not a MapRecordEnum"));
+    }
+  }
+
+  private GenericData.Record buildComplexRecord() throws IOException {
+
+    Schema schema = new Schema.Parser().parse(new File("../../../share/test/schemas/RecordWithRequiredFields.avsc"));
+
+    GenericData.Record topLevelRecord = new GenericData.Record(schema);
+    GenericData.Record unionRecord = new GenericData.Record(schema.getField("unionField").schema().getTypes().get(1));
+    Schema arraySchema = unionRecord.getSchema().getField("arrayField").schema();
+    GenericData.Record arrayRecord1 = new GenericData.Record(arraySchema.getElementType());
+    GenericData.Record arrayRecord2 = new GenericData.Record(arraySchema.getElementType());
+    GenericData.Array<GenericData.Record> array = new GenericData.Array<>(arraySchema,
+        Arrays.asList(arrayRecord1, arrayRecord2));
+    Schema mapRecordSchema = arraySchema.getElementType().getField("mapField").schema().getValueType();
+    GenericData.Record mapRecordA = new GenericData.Record(mapRecordSchema);
+    Schema mapRecordEnumSchema = mapRecordSchema.getField("enumField").schema();
+
+    mapRecordA.put("enumField", new GenericData.EnumSymbol(mapRecordEnumSchema, "B"));
+    mapRecordA.put("strField", "4");
+
+    arrayRecord1.put("strField", "2");
+    HashMap<String, GenericData.Record> map1 = new HashMap<>();
+    map1.put("a", mapRecordA);
+    arrayRecord1.put("mapField", map1);
+
+    arrayRecord2.put("strField", "2");
+    HashMap<String, GenericData.Record> map2 = new HashMap<>();
+    map2.put("a", mapRecordA);
+    arrayRecord2.put("mapField", map2);
+
+    unionRecord.put(unionRecord.getSchema().getField("strField").pos(), "1");
+    unionRecord.put(unionRecord.getSchema().getField("arrayField").pos(), array); // BOOM
+
+    topLevelRecord.put(topLevelRecord.getSchema().getField("strField").pos(), "0");
+    topLevelRecord.put(topLevelRecord.getSchema().getField("unionField").pos(), unionRecord);
+
+    return topLevelRecord;
   }
 
   private Schema schemaWithExplicitNullDefault() {
@@ -347,10 +475,15 @@ public class TestGenericDatumWriter {
     return new Schema.Parser().parse(schema);
   }
 
-  private void writeObject(Schema schema, GenericRecord datum) throws Exception {
+  private void writeObject(GenericRecord datum) throws Exception {
+    writeObject(datum.getSchema(), datum);
+  }
+
+  private void writeObject(Schema schema, Object datum) throws Exception {
     BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(new ByteArrayOutputStream(), null);
-    GenericDatumWriter<GenericData.Record> writer = new GenericDatumWriter<>(schema);
-    writer.write(schema, datum, encoder);
+    GenericDatumWriter<Object> writer = new GenericDatumWriter<>(schema);
+    writer.write(datum, encoder);
+    encoder.flush();
   }
 
   private GenericRecord createRecordWithDefaultField(Schema schema) {
diff --git a/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificDatumWriter.java b/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificDatumWriter.java
index 989fb15..ed4d140 100644
--- a/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificDatumWriter.java
+++ b/lang/java/ipc/src/test/java/org/apache/avro/specific/TestSpecificDatumWriter.java
@@ -20,13 +20,25 @@ package org.apache.avro.specific;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.avro.ArrayRecord;
+import org.apache.avro.AvroTypeException;
+import org.apache.avro.MapRecord;
+import org.apache.avro.MapRecordEnum;
+import org.apache.avro.RecordWithRequiredFields;
 import org.apache.avro.Schema;
+import org.apache.avro.UnionRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.io.JsonEncoder;
 import org.apache.avro.test.Kind;
 import org.apache.avro.test.MD5;
 import org.apache.avro.test.TestRecordWithUnion;
 import org.apache.avro.test.TestRecord;
+import org.junit.Assert;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -69,9 +81,127 @@ public class TestSpecificDatumWriter {
       writer.write(testRecord, encoder);
       fail("Exception not thrown");
     } catch (NullPointerException e) {
-      assertTrue(e.getMessage().contains("null of string in field 'name'"));
+      assertTrue(e.getMessage().contains("null value for (non-nullable) string at TestRecord.name"));
     } finally {
       out.close();
     }
   }
+
+  @Test
+  public void testNestedNPEErrorClarity() throws Exception {
+    RecordWithRequiredFields topLevelRecord = buildComplexRecord();
+    topLevelRecord.getUnionField().getArrayField().get(0).getMapField().get("a").setStrField(null);
+    try {
+      writeObject(topLevelRecord, false);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField[\"a\"].strField"));
+    }
+  }
+
+  @Test
+  public void testNestedNPEErrorClarityWithCustomCoders() throws Exception {
+    RecordWithRequiredFields topLevelRecord = buildComplexRecord();
+    topLevelRecord.getUnionField().getArrayField().get(0).getMapField().get("a").setEnumField(null);
+    try {
+      writeObject(topLevelRecord, true);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(),
+          expected.getMessage().contains("custom coders were used"));
+    }
+  }
+
+  @Test
+  public void testNPEForMapKeyErrorClarity() throws Exception {
+    RecordWithRequiredFields topLevelRecord = buildComplexRecord();
+    Map<String, MapRecord> map = topLevelRecord.getUnionField().getArrayField().get(0).getMapField();
+    map.put(null, map.get("a")); // value is valid, but key is null
+    try {
+      writeObject(topLevelRecord, false);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("null key in map at RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField"));
+    }
+  }
+
+  @Test
+  public void testNPEForMapKeyErrorClarityWithCustomCoders() throws Exception {
+    RecordWithRequiredFields topLevelRecord = buildComplexRecord();
+    Map<String, MapRecord> map = topLevelRecord.getUnionField().getArrayField().get(0).getMapField();
+    map.put(null, map.get("a")); // value is valid, but key is null
+    try {
+      writeObject(topLevelRecord, true);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(),
+          expected.getMessage().contains("custom coders were used"));
+    }
+  }
+
+  @Test
+  public void testNestedATEErrorClarity() throws Exception {
+    RecordWithRequiredFields topLevelRecord = buildComplexRecord();
+    topLevelRecord.getUnionField().getArrayField().get(0).getMapField().get("a").setEnumField(null); // not an enum
+    try {
+      writeObject(topLevelRecord, false);
+      Assert.fail("expected to throw");
+    } catch (AvroTypeException expected) {
+      Assert.assertTrue("unexpected message " + expected.getMessage(), expected.getMessage()
+          .contains("RecordWithRequiredFields.unionField[UnionRecord].arrayField[0].mapField[\"a\"].enumField"));
+    }
+  }
+
+  @Test
+  public void testNestedATEErrorClarityWithCustomCoders() throws Exception {
+    RecordWithRequiredFields topLevelRecord = buildComplexRecord();
+    topLevelRecord.getUnionField().getArrayField().get(0).getMapField().get("a").setEnumField(null); // not an enum
+    try {
+      writeObject(topLevelRecord, true);
+      Assert.fail("expected to throw");
+    } catch (NullPointerException expected) {
+      // with custom coders this gets us an NPE ...
+      Assert.assertTrue("unexpected message " + expected.getMessage(),
+          expected.getMessage().contains("custom coders were used"));
+    }
+  }
+
+  private RecordWithRequiredFields buildComplexRecord() {
+    RecordWithRequiredFields topLevelRecord = new RecordWithRequiredFields();
+    UnionRecord unionRecord = new UnionRecord();
+    ArrayRecord arrayRecord1 = new ArrayRecord();
+    ArrayRecord arrayRecord2 = new ArrayRecord();
+    MapRecord mapRecordA = new MapRecord();
+    mapRecordA.setEnumField(MapRecordEnum.B);
+    mapRecordA.setStrField("4");
+    arrayRecord1.setStrField("2");
+    Map<String, MapRecord> map1 = new HashMap<>();
+    map1.put("a", mapRecordA);
+    arrayRecord1.setMapField(map1);
+    arrayRecord2.setStrField("2");
+    Map<String, MapRecord> map2 = new HashMap<>();
+    map2.put("a", mapRecordA);
+    arrayRecord2.setMapField(map2);
+    unionRecord.setStrField("1");
+    unionRecord.setArrayField(Arrays.asList(arrayRecord1, arrayRecord2));
+    topLevelRecord.setStrField("0");
+    topLevelRecord.setUnionField(unionRecord);
+
+    return topLevelRecord;
+  }
+
+  private void writeObject(IndexedRecord datum, boolean useCustomCoders) throws Exception {
+    writeObject(datum.getSchema(), datum, useCustomCoders);
+  }
+
+  private void writeObject(Schema schema, Object datum, boolean useCustomCoders) throws Exception {
+    BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(new ByteArrayOutputStream(), null);
+    SpecificData specificData = new SpecificData();
+    specificData.setCustomCoders(useCustomCoders);
+    SpecificDatumWriter<Object> writer = new SpecificDatumWriter<>(schema, specificData);
+    writer.write(datum, encoder);
+    encoder.flush();
+  }
 }
diff --git a/share/test/schemas/RecordWithRequiredFields.avsc b/share/test/schemas/RecordWithRequiredFields.avsc
new file mode 100644
index 0000000..2292c58
--- /dev/null
+++ b/share/test/schemas/RecordWithRequiredFields.avsc
@@ -0,0 +1,53 @@
+{
+  "type" : "record",
+  "name" : "RecordWithRequiredFields",
+  "namespace" : "org.apache.avro",
+  "fields" : [ {
+    "name" : "strField",
+    "type" : "string"
+  }, {
+    "name" : "unionField",
+    "type" : [ "null", {
+      "type" : "record",
+      "name" : "UnionRecord",
+      "fields" : [ {
+        "name" : "strField",
+        "type" : "string"
+      }, {
+        "name" : "arrayField",
+        "type" : {
+          "type" : "array",
+          "items" : {
+            "type" : "record",
+            "name" : "ArrayRecord",
+            "fields" : [ {
+              "name" : "strField",
+              "type" : "string"
+            }, {
+              "name" : "mapField",
+              "type" : {
+                "type" : "map",
+                "values" : {
+                  "type" : "record",
+                  "name" : "MapRecord",
+                  "fields" : [ {
+                    "name" : "enumField",
+                    "type" : {
+                      "type" : "enum",
+                      "name" : "MapRecordEnum",
+                      "symbols" : [ "A", "B" ],
+                      "default" : "A"
+                    }
+                  }, {
+                    "name" : "strField",
+                    "type" : "string"
+                  } ]
+                }
+              }
+            } ]
+          }
+        }
+      } ]
+    } ]
+  } ]
+}