You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by th...@apache.org on 2016/11/07 19:59:00 UTC

[10/50] incubator-beam git commit: [BEAM-790] Validate PipelineOptions Default annotation

[BEAM-790] Validate PipelineOptions Default annotation


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/f605b02a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/f605b02a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/f605b02a

Branch: refs/heads/apex-runner
Commit: f605b02a72258aee20d607e54a443113bfb762ad
Parents: 6a05cf4
Author: Pei He <pe...@google.com>
Authored: Thu Oct 20 18:07:35 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Thu Nov 3 11:38:42 2016 -0700

----------------------------------------------------------------------
 .../sdk/options/PipelineOptionsFactory.java     | 248 ++++++++++++++-----
 .../sdk/options/ProxyInvocationHandler.java     |   5 +-
 .../beam/sdk/util/common/ReflectHelpers.java    |  17 ++
 .../sdk/options/PipelineOptionsFactoryTest.java | 239 ++++++++++++++++++
 .../sdk/options/ProxyInvocationHandlerTest.java |  29 ++-
 .../sdk/util/common/ReflectHelpersTest.java     |  25 ++
 6 files changed, 491 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f605b02a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
index eefe8c7..304e166 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
@@ -1057,56 +1057,130 @@ public class PipelineOptionsFactory {
       methodNameToAllMethodMap.put(method, method);
     }
 
-    List<InconsistentlyIgnoredGetters> incompletelyIgnoredGetters = new ArrayList<>();
-    List<IgnoredSetter> ignoredSetters = new ArrayList<>();
+    // Verify that there is no getter with a mixed @JsonIgnore annotation.
+    validateGettersHaveConsistentAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_IGNORE);
 
-    for (PropertyDescriptor descriptor : descriptors) {
+    // Verify that there is no getter with a mixed @Default annotation.
+    validateGettersHaveConsistentAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.DEFAULT_VALUE);
+
+    // Verify that no setter has @JsonIgnore.
+    validateSettersDoNotHaveAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_IGNORE);
+
+    // Verify that no setter has @Default.
+    validateSettersDoNotHaveAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.DEFAULT_VALUE);
+  }
+
+  /**
+   * Validates that getters don't have mixed annotation.
+   */
+  private static void validateGettersHaveConsistentAnnotation(
+      SortedSetMultimap<Method, Method> methodNameToAllMethodMap,
+      List<PropertyDescriptor> descriptors,
+      final AnnotationPredicates annotationPredicates) {
+    List<InconsistentlyAnnotatedGetters> inconsistentlyAnnotatedGetters = new ArrayList<>();
+    for (final PropertyDescriptor descriptor : descriptors) {
       if (descriptor.getReadMethod() == null
-          || descriptor.getWriteMethod() == null
-          || IGNORED_METHODS.contains(descriptor.getReadMethod())
-          || IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
+          || IGNORED_METHODS.contains(descriptor.getReadMethod())) {
         continue;
       }
-      // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
-      // that no setter has @JsonIgnore.
+
       SortedSet<Method> getters = methodNameToAllMethodMap.get(descriptor.getReadMethod());
-      SortedSet<Method> gettersWithJsonIgnore = Sets.filter(getters, JsonIgnorePredicate.INSTANCE);
+      SortedSet<Method> gettersWithTheAnnotation =
+          Sets.filter(getters, annotationPredicates.forMethod);
+      Set<Annotation> distinctAnnotations = Sets.newLinkedHashSet(FluentIterable
+          .from(gettersWithTheAnnotation)
+          .transformAndConcat(new Function<Method, Iterable<? extends Annotation>>() {
+            @Nonnull
+            @Override
+            public Iterable<? extends Annotation> apply(@Nonnull Method method) {
+              return FluentIterable.of(method.getAnnotations());
+            }
+          })
+          .filter(annotationPredicates.forAnnotation));
+
+
+      if (distinctAnnotations.size() > 1) {
+        throw new IllegalArgumentException(String.format(
+            "Property [%s] is marked with contradictory annotations. Found [%s].",
+            descriptor.getName(),
+            FluentIterable.from(gettersWithTheAnnotation)
+                .transformAndConcat(new Function<Method, Iterable<String>>() {
+                  @Nonnull
+                  @Override
+                  public Iterable<String> apply(final @Nonnull Method method) {
+                    return FluentIterable.of(method.getAnnotations())
+                        .filter(annotationPredicates.forAnnotation)
+                        .transform(new Function<Annotation, String>() {
+                          @Nonnull
+                          @Override
+                          public String apply(@Nonnull Annotation annotation) {
+                            return String.format(
+                                "[%s on %s]",
+                                ReflectHelpers.ANNOTATION_FORMATTER.apply(annotation),
+                                ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(method));
+                          }
+                        });
+
+                  }
+                })
+                .join(Joiner.on(", "))));
+      }
 
       Iterable<String> getterClassNames = FluentIterable.from(getters)
           .transform(MethodToDeclaringClassFunction.INSTANCE)
           .transform(ReflectHelpers.CLASS_NAME);
-      Iterable<String> gettersWithJsonIgnoreClassNames = FluentIterable.from(gettersWithJsonIgnore)
+      Iterable<String> gettersWithTheAnnotationClassNames =
+          FluentIterable.from(gettersWithTheAnnotation)
           .transform(MethodToDeclaringClassFunction.INSTANCE)
           .transform(ReflectHelpers.CLASS_NAME);
 
-      if (!(gettersWithJsonIgnore.isEmpty() || getters.size() == gettersWithJsonIgnore.size())) {
-        InconsistentlyIgnoredGetters err = new InconsistentlyIgnoredGetters();
+      if (!(gettersWithTheAnnotation.isEmpty()
+            || getters.size() == gettersWithTheAnnotation.size())) {
+        InconsistentlyAnnotatedGetters err = new InconsistentlyAnnotatedGetters();
         err.descriptor = descriptor;
         err.getterClassNames = getterClassNames;
-        err.gettersWithJsonIgnoreClassNames = gettersWithJsonIgnoreClassNames;
-        incompletelyIgnoredGetters.add(err);
+        err.gettersWithTheAnnotationClassNames = gettersWithTheAnnotationClassNames;
+        inconsistentlyAnnotatedGetters.add(err);
       }
-      if (!incompletelyIgnoredGetters.isEmpty()) {
+    }
+    throwForGettersWithInconsistentAnnotation(
+        inconsistentlyAnnotatedGetters, annotationPredicates.annotationClass);
+  }
+
+  /**
+   * Validates that setters don't have the given annotation.
+   */
+  private static void validateSettersDoNotHaveAnnotation(
+      SortedSetMultimap<Method, Method> methodNameToAllMethodMap,
+      List<PropertyDescriptor> descriptors,
+      AnnotationPredicates annotationPredicates) {
+    List<AnnotatedSetter> annotatedSetters = new ArrayList<>();
+    for (PropertyDescriptor descriptor : descriptors) {
+      if (descriptor.getWriteMethod() == null
+          || IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
         continue;
       }
+      SortedSet<Method> settersWithTheAnnotation = Sets.filter(
+          methodNameToAllMethodMap.get(descriptor.getWriteMethod()),
+          annotationPredicates.forMethod);
 
-      SortedSet<Method> settersWithJsonIgnore =
-          Sets.filter(methodNameToAllMethodMap.get(descriptor.getWriteMethod()),
-              JsonIgnorePredicate.INSTANCE);
-
-      Iterable<String> settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore)
+      Iterable<String> settersWithTheAnnotationClassNames =
+          FluentIterable.from(settersWithTheAnnotation)
           .transform(MethodToDeclaringClassFunction.INSTANCE)
           .transform(ReflectHelpers.CLASS_NAME);
 
-      if (!settersWithJsonIgnore.isEmpty()) {
-        IgnoredSetter ignored = new IgnoredSetter();
-        ignored.descriptor = descriptor;
-        ignored.settersWithJsonIgnoreClassNames = settersWithJsonIgnoreClassNames;
-        ignoredSetters.add(ignored);
+      if (!settersWithTheAnnotation.isEmpty()) {
+        AnnotatedSetter annotated = new AnnotatedSetter();
+        annotated.descriptor = descriptor;
+        annotated.settersWithTheAnnotationClassNames = settersWithTheAnnotationClassNames;
+        annotatedSetters.add(annotated);
       }
     }
-    throwForGettersWithInconsistentJsonIgnore(incompletelyIgnoredGetters);
-    throwForSettersWithJsonIgnore(ignoredSetters);
+    throwForSettersWithTheAnnotation(annotatedSetters, annotationPredicates.annotationClass);
   }
 
   /**
@@ -1221,53 +1295,62 @@ public class PipelineOptionsFactory {
     }
   }
 
-  private static class InconsistentlyIgnoredGetters {
+  private static class InconsistentlyAnnotatedGetters {
     PropertyDescriptor descriptor;
     Iterable<String> getterClassNames;
-    Iterable<String> gettersWithJsonIgnoreClassNames;
+    Iterable<String> gettersWithTheAnnotationClassNames;
   }
 
-  private static void throwForGettersWithInconsistentJsonIgnore(
-      List<InconsistentlyIgnoredGetters> getters) {
+  private static void throwForGettersWithInconsistentAnnotation(
+      List<InconsistentlyAnnotatedGetters> getters,
+      Class<? extends Annotation> annotationClass) {
     if (getters.size() == 1) {
-      InconsistentlyIgnoredGetters getter = getters.get(0);
+      InconsistentlyAnnotatedGetters getter = getters.get(0);
       throw new IllegalArgumentException(String.format(
-          "Expected getter for property [%s] to be marked with @JsonIgnore on all %s, "
+          "Expected getter for property [%s] to be marked with @%s on all %s, "
           + "found only on %s",
-          getter.descriptor.getName(), getter.getterClassNames,
-          getter.gettersWithJsonIgnoreClassNames));
+          getter.descriptor.getName(),
+          annotationClass.getSimpleName(),
+          getter.getterClassNames,
+          getter.gettersWithTheAnnotationClassNames));
     } else if (getters.size() > 1) {
-      StringBuilder errorBuilder =
-          new StringBuilder("Property getters are inconsistently marked with @JsonIgnore:");
-      for (InconsistentlyIgnoredGetters getter : getters) {
+      StringBuilder errorBuilder = new StringBuilder(String.format(
+          "Property getters are inconsistently marked with @%s:", annotationClass.getSimpleName()));
+      for (InconsistentlyAnnotatedGetters getter : getters) {
         errorBuilder.append(
             String.format("%n  - Expected for property [%s] to be marked on all %s, "
                 + "found only on %s",
                 getter.descriptor.getName(), getter.getterClassNames,
-                getter.gettersWithJsonIgnoreClassNames));
+                getter.gettersWithTheAnnotationClassNames));
       }
       throw new IllegalArgumentException(errorBuilder.toString());
     }
   }
 
-  private static class IgnoredSetter {
+  private static class AnnotatedSetter {
     PropertyDescriptor descriptor;
-    Iterable<String> settersWithJsonIgnoreClassNames;
+    Iterable<String> settersWithTheAnnotationClassNames;
   }
 
-  private static void throwForSettersWithJsonIgnore(List<IgnoredSetter> setters) {
+  private static void throwForSettersWithTheAnnotation(
+      List<AnnotatedSetter> setters,
+      Class<? extends Annotation> annotationClass) {
     if (setters.size() == 1) {
-      IgnoredSetter setter = setters.get(0);
-      throw new IllegalArgumentException(
-          String.format("Expected setter for property [%s] to not be marked with @JsonIgnore on %s",
-              setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames));
+      AnnotatedSetter setter = setters.get(0);
+      throw new IllegalArgumentException(String.format(
+          "Expected setter for property [%s] to not be marked with @%s on %s",
+          setter.descriptor.getName(),
+          annotationClass.getSimpleName(),
+          setter.settersWithTheAnnotationClassNames));
     } else if (setters.size() > 1) {
-      StringBuilder builder = new StringBuilder("Found setters marked with @JsonIgnore:");
-      for (IgnoredSetter setter : setters) {
-        builder.append(
-            String.format("%n  - Setter for property [%s] should not be marked with @JsonIgnore "
-                + "on %s",
-                setter.descriptor.getName(), setter.settersWithJsonIgnoreClassNames));
+      StringBuilder builder = new StringBuilder(
+          String.format("Found setters marked with @%s:", annotationClass.getSimpleName()));
+      for (AnnotatedSetter setter : setters) {
+        builder.append(String.format(
+            "%n  - Setter for property [%s] should not be marked with @%s on %s",
+            setter.descriptor.getName(),
+            annotationClass.getSimpleName(),
+            setter.settersWithTheAnnotationClassNames));
       }
       throw new IllegalArgumentException(builder.toString());
     }
@@ -1353,14 +1436,61 @@ public class PipelineOptionsFactory {
   }
 
   /**
-   * A {@link Predicate} that returns true if the method is annotated with
-   * {@link JsonIgnore @JsonIgnore}.
+   * A {@link Predicate} that returns true if the method is annotated with {@code annotationClass}.
    */
-  static class JsonIgnorePredicate implements Predicate<Method> {
-    static final JsonIgnorePredicate INSTANCE = new JsonIgnorePredicate();
-    @Override
-    public boolean apply(Method input) {
-      return input.isAnnotationPresent(JsonIgnore.class);
+  static class AnnotationPredicates {
+    static final AnnotationPredicates JSON_IGNORE = new AnnotationPredicates(
+        JsonIgnore.class,
+        new Predicate<Annotation>() {
+          @Override
+          public boolean apply(@Nonnull Annotation input) {
+            return JsonIgnore.class.equals(input.annotationType());
+          }
+        },
+        new Predicate<Method>() {
+          @Override
+          public boolean apply(@Nonnull Method input) {
+            return input.isAnnotationPresent(JsonIgnore.class);
+          }});
+
+    private static final Set<Class<?>> DEFAULT_ANNOTATION_CLASSES = Sets.newHashSet(
+        FluentIterable.of(Default.class.getDeclaredClasses())
+        .filter(new Predicate<Class<?>>() {
+          @Override
+          public boolean apply(@Nonnull Class<?> klass) {
+            return klass.isAnnotation();
+          }}));
+
+    static final AnnotationPredicates DEFAULT_VALUE = new AnnotationPredicates(
+        Default.class,
+        new Predicate<Annotation>() {
+          @Override
+          public boolean apply(@Nonnull Annotation input) {
+            return DEFAULT_ANNOTATION_CLASSES.contains(input.annotationType());
+          }
+        },
+        new Predicate<Method> () {
+          @Override
+          public boolean apply(@Nonnull Method input) {
+            for (Annotation annotation : input.getAnnotations()) {
+              if (DEFAULT_ANNOTATION_CLASSES.contains(annotation.annotationType())) {
+                return true;
+              }
+            }
+            return false;
+          }});
+
+    final Class<? extends Annotation> annotationClass;
+    final Predicate<Annotation> forAnnotation;
+    final Predicate<Method> forMethod;
+
+    AnnotationPredicates(
+        Class<? extends Annotation> annotationClass,
+        Predicate<Annotation> forAnnotation,
+        Predicate<Method> forMethod) {
+      this.annotationClass = annotationClass;
+      this.forAnnotation = forAnnotation;
+      this.forMethod = forMethod;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f605b02a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index f7c6da2..86f9918 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -63,7 +63,7 @@ import java.util.TreeMap;
 import java.util.concurrent.ThreadLocalRandom;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
-import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate;
+import org.apache.beam.sdk.options.PipelineOptionsFactory.AnnotationPredicates;
 import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration;
 import org.apache.beam.sdk.options.ValueProvider.RuntimeValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
@@ -638,7 +638,8 @@ class ProxyInvocationHandler implements InvocationHandler {
       // Find all the method names that are annotated with JSON ignore.
       Set<String> jsonIgnoreMethodNames = FluentIterable.from(
           ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces))
-          .filter(JsonIgnorePredicate.INSTANCE).transform(new Function<Method, String>() {
+          .filter(AnnotationPredicates.JSON_IGNORE.forMethod)
+          .transform(new Function<Method, String>() {
             @Override
             public String apply(Method input) {
               return input.getName();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f605b02a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
index 2d92162..2b08fee 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
@@ -26,6 +26,7 @@ import com.google.common.base.Joiner;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Queues;
+import java.lang.annotation.Annotation;
 import java.lang.reflect.GenericArrayType;
 import java.lang.reflect.Method;
 import java.lang.reflect.ParameterizedType;
@@ -86,6 +87,22 @@ public class ReflectHelpers {
     }
   };
 
+  /**
+   * A {@link Function} that returns a concise string for a {@link Annotation}.
+   */
+  public static final Function<Annotation, String> ANNOTATION_FORMATTER =
+      new Function<Annotation, String>() {
+        @Override
+        public String apply(@Nonnull Annotation annotation) {
+          String annotationName = annotation.annotationType().getName();
+          String annotationNameWithoutPackage =
+              annotationName.substring(annotationName.lastIndexOf('.') + 1).replace('$', '.');
+          String annotationToString = annotation.toString();
+          String values = annotationToString.substring(annotationToString.indexOf('('));
+          return String.format("%s%s", annotationNameWithoutPackage, values);
+        }
+      };
+
   /** A {@link Function} that formats types. */
   public static final Function<Type, String> TYPE_SIMPLE_DESCRIPTION =
       new Function<Type, String>() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f605b02a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
index 64d8c51..0a2324f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
@@ -369,6 +369,16 @@ public class PipelineOptionsFactoryTest {
     void setObject(Object value);
   }
 
+  /**
+   * This class is has a conflicting {@link JsonIgnore @JsonIgnore} value with
+   * {@link GetterWithJsonIgnore}.
+   */
+  public interface GetterWithInconsistentJsonIgnoreValue extends PipelineOptions {
+    @JsonIgnore(value = false)
+    Object getObject();
+    void setObject(Object value);
+  }
+
   @Test
   public void testNotAllGettersAnnotatedWithJsonIgnore() throws Exception {
     // Initial construction is valid.
@@ -441,6 +451,235 @@ public class PipelineOptionsFactoryTest {
     options.as(MultipleGettersWithInconsistentJsonIgnore.class);
   }
 
+  /** Test interface that has {@link Default @Default} on a setter for a property. */
+  public interface SetterWithDefault extends PipelineOptions {
+    String getValue();
+    @Default.String("abc")
+    void setValue(String value);
+  }
+
+  @Test
+  public void testSetterAnnotatedWithDefault() throws Exception {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Expected setter for property [value] to not be marked with @Default on ["
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$SetterWithDefault]");
+    PipelineOptionsFactory.as(SetterWithDefault.class);
+  }
+
+  /** Test interface that has {@link Default @Default} on multiple setters. */
+  public interface MultiSetterWithDefault extends SetterWithDefault {
+    Integer getOther();
+    @Default.String("abc")
+    void setOther(Integer other);
+  }
+
+  @Test
+  public void testMultipleSettersAnnotatedWithDefault() throws Exception {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage("Found setters marked with @Default:");
+    expectedException.expectMessage(
+        "property [other] should not be marked with @Default on ["
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MultiSetterWithDefault]");
+    expectedException.expectMessage(
+        "property [value] should not be marked with @Default on ["
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$SetterWithDefault]");
+    PipelineOptionsFactory.as(MultiSetterWithDefault.class);
+  }
+
+  /**
+   * This class is has a conflicting field with {@link CombinedObject} that doesn't have
+   * {@link Default @Default}.
+   */
+  private interface GetterWithDefault extends PipelineOptions {
+    @Default.Integer(1)
+    Object getObject();
+    void setObject(Object value);
+  }
+
+  /**
+   * This class is consistent with {@link GetterWithDefault} that has the same
+   * {@link Default @Default}.
+   */
+  private interface GetterWithConsistentDefault extends PipelineOptions {
+    @Default.Integer(1)
+    Object getObject();
+    void setObject(Object value);
+  }
+
+  /**
+   * This class is inconsistent with {@link GetterWithDefault} that has a different
+   * {@link Default @Default}.
+   */
+  private interface GetterWithInconsistentDefaultType extends PipelineOptions {
+    @Default.String("abc")
+    Object getObject();
+    void setObject(Object value);
+  }
+
+  /**
+   * This class is inconsistent with {@link GetterWithDefault} that has a different
+   * {@link Default @Default} value.
+   */
+  private interface GetterWithInconsistentDefaultValue extends PipelineOptions {
+    @Default.Integer(0)
+    Object getObject();
+    void setObject(Object value);
+  }
+
+  @Test
+  public void testNotAllGettersAnnotatedWithDefault() throws Exception {
+    // Initial construction is valid.
+    GetterWithDefault options = PipelineOptionsFactory.as(GetterWithDefault.class);
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Expected getter for property [object] to be marked with @Default on all ["
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$GetterWithDefault, "
+            + "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MissingSetter], "
+            + "found only on [org.apache.beam.sdk.options."
+            + "PipelineOptionsFactoryTest$GetterWithDefault]");
+
+    // When we attempt to convert, we should error at this moment.
+    options.as(CombinedObject.class);
+  }
+
+  @Test
+  public void testGettersAnnotatedWithConsistentDefault() throws Exception {
+    GetterWithConsistentDefault options = PipelineOptionsFactory
+        .as(GetterWithDefault.class)
+        .as(GetterWithConsistentDefault.class);
+
+    assertEquals(1, options.getObject());
+  }
+
+  @Test
+  public void testGettersAnnotatedWithInconsistentDefault() throws Exception {
+    // Initial construction is valid.
+    GetterWithDefault options = PipelineOptionsFactory.as(GetterWithDefault.class);
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Property [object] is marked with contradictory annotations. Found ["
+            + "[Default.Integer(value=1) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GetterWithDefault#getObject()], "
+            + "[Default.String(value=abc) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GetterWithInconsistentDefaultType#getObject()]].");
+
+    // When we attempt to convert, we should error at this moment.
+    options.as(GetterWithInconsistentDefaultType.class);
+  }
+
+  @Test
+  public void testGettersAnnotatedWithInconsistentDefaultValue() throws Exception {
+    // Initial construction is valid.
+    GetterWithDefault options = PipelineOptionsFactory.as(GetterWithDefault.class);
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Property [object] is marked with contradictory annotations. Found ["
+            + "[Default.Integer(value=1) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GetterWithDefault#getObject()], "
+            + "[Default.Integer(value=0) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GetterWithInconsistentDefaultValue#getObject()]].");
+
+    // When we attempt to convert, we should error at this moment.
+    options.as(GetterWithInconsistentDefaultValue.class);
+  }
+
+  @Test
+  public void testGettersAnnotatedWithInconsistentJsonIgnoreValue() throws Exception {
+    // Initial construction is valid.
+    GetterWithJsonIgnore options = PipelineOptionsFactory.as(GetterWithJsonIgnore.class);
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Property [object] is marked with contradictory annotations. Found ["
+            + "[JsonIgnore(value=false) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GetterWithInconsistentJsonIgnoreValue#getObject()], "
+            + "[JsonIgnore(value=true) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GetterWithJsonIgnore#getObject()]].");
+
+    // When we attempt to convert, we should error at this moment.
+    options.as(GetterWithInconsistentJsonIgnoreValue.class);
+  }
+
+  private interface GettersWithMultipleDefault extends PipelineOptions {
+    @Default.String("abc")
+    @Default.Integer(0)
+    Object getObject();
+    void setObject(Object value);
+  }
+
+  @Test
+  public void testGettersWithMultipleDefaults() throws Exception {
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+        "Property [object] is marked with contradictory annotations. Found ["
+            + "[Default.String(value=abc) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GettersWithMultipleDefault#getObject()], "
+            + "[Default.Integer(value=0) on org.apache.beam.sdk.options.PipelineOptionsFactoryTest"
+            + "$GettersWithMultipleDefault#getObject()]].");
+
+    // When we attempt to create, we should error at this moment.
+    PipelineOptionsFactory.as(GettersWithMultipleDefault.class);
+  }
+
+  private interface MultiGettersWithDefault extends PipelineOptions {
+    Object getObject();
+    void setObject(Object value);
+
+    @Default.Integer(1)
+    Integer getOther();
+    void setOther(Integer value);
+
+    Void getConsistent();
+    void setConsistent(Void consistent);
+  }
+
+  private interface MultipleGettersWithInconsistentDefault extends PipelineOptions {
+    @Default.Boolean(true)
+    Object getObject();
+    void setObject(Object value);
+
+    Integer getOther();
+    void setOther(Integer value);
+
+    Void getConsistent();
+    void setConsistent(Void consistent);
+  }
+
+  @Test
+  public void testMultipleGettersWithInconsistentDefault() {
+    // Initial construction is valid.
+    MultiGettersWithDefault options = PipelineOptionsFactory.as(MultiGettersWithDefault.class);
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage("Property getters are inconsistently marked with @Default:");
+    expectedException.expectMessage(
+        "property [object] to be marked on all");
+    expectedException.expectMessage("found only on [org.apache.beam.sdk.options."
+        + "PipelineOptionsFactoryTest$MultiGettersWithDefault]");
+    expectedException.expectMessage(
+        "property [other] to be marked on all");
+    expectedException.expectMessage("found only on [org.apache.beam.sdk.options."
+        + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentDefault]");
+
+    expectedException.expectMessage(Matchers.anyOf(
+        containsString(java.util.Arrays.toString(new String[]
+            {"org.apache.beam.sdk.options."
+                + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentDefault",
+                "org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MultiGettersWithDefault"})),
+        containsString(java.util.Arrays.toString(new String[]
+            {"org.apache.beam.sdk.options.PipelineOptionsFactoryTest$MultiGettersWithDefault",
+                "org.apache.beam.sdk.options."
+                    + "PipelineOptionsFactoryTest$MultipleGettersWithInconsistentDefault"}))));
+    expectedException.expectMessage(not(containsString("property [consistent]")));
+
+    // When we attempt to convert, we should error immediately
+    options.as(MultipleGettersWithInconsistentDefault.class);
+  }
+
   @Test
   public void testAppNameIsNotOverriddenWhenPassedInViaCommandLine() {
     ApplicationNameOptions options = PipelineOptionsFactory

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f605b02a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
index 52b98ee..5e97eed 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
@@ -235,38 +235,45 @@ public class ProxyInvocationHandlerTest {
         .testEquals();
   }
 
+  /** A test interface for string with default. */
+  public interface StringWithDefault extends PipelineOptions {
+    @Default.String("testString")
+    String getString();
+    void setString(String value);
+  }
+
   @Test
   public void testToString() throws Exception {
     ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.<String, Object>newHashMap());
-    Simple proxy = handler.as(Simple.class);
+    StringWithDefault proxy = handler.as(StringWithDefault.class);
     proxy.setString("stringValue");
     DefaultAnnotations proxy2 = proxy.as(DefaultAnnotations.class);
     proxy2.setLong(57L);
     assertEquals("Current Settings:\n"
-        + "  long: 57\n"
-        + "  string: stringValue\n",
+            + "  long: 57\n"
+            + "  string: stringValue\n",
         proxy.toString());
   }
 
   @Test
   public void testToStringAfterDeserializationContainsJsonEntries() throws Exception {
     ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.<String, Object>newHashMap());
-    Simple proxy = handler.as(Simple.class);
+    StringWithDefault proxy = handler.as(StringWithDefault.class);
     Long optionsId = proxy.getOptionsId();
     proxy.setString("stringValue");
     DefaultAnnotations proxy2 = proxy.as(DefaultAnnotations.class);
     proxy2.setLong(57L);
     assertEquals(String.format("Current Settings:\n"
-        + "  long: 57\n"
-        + "  optionsId: %d\n"
-        + "  string: \"stringValue\"\n", optionsId),
+            + "  long: 57\n"
+            + "  optionsId: %d\n"
+            + "  string: \"stringValue\"\n", optionsId),
         serializeDeserialize(PipelineOptions.class, proxy2).toString());
   }
 
   @Test
   public void testToStringAfterDeserializationContainsOverriddenEntries() throws Exception {
     ProxyInvocationHandler handler = new ProxyInvocationHandler(Maps.<String, Object>newHashMap());
-    Simple proxy = handler.as(Simple.class);
+    StringWithDefault proxy = handler.as(StringWithDefault.class);
     Long optionsId = proxy.getOptionsId();
     proxy.setString("stringValue");
     DefaultAnnotations proxy2 = proxy.as(DefaultAnnotations.class);
@@ -274,9 +281,9 @@ public class ProxyInvocationHandlerTest {
     Simple deserializedOptions = serializeDeserialize(Simple.class, proxy2);
     deserializedOptions.setString("overriddenValue");
     assertEquals(String.format("Current Settings:\n"
-        + "  long: 57\n"
-        + "  optionsId: %d\n"
-        + "  string: overriddenValue\n", optionsId),
+            + "  long: 57\n"
+            + "  optionsId: %d\n"
+            + "  string: overriddenValue\n", optionsId),
         deserializedOptions.toString());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f605b02a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
index e1073da..8a1708c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
@@ -19,8 +19,11 @@ package org.apache.beam.sdk.util.common;
 
 import static org.junit.Assert.assertEquals;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import java.util.List;
 import java.util.Map;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.values.TypeDescriptor;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -123,4 +126,26 @@ public class ReflectHelpersTest {
         ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(
             new TypeDescriptor<Map<? super InputT, ? extends OutputT>>() {}.getType()));
   }
+
+  private interface Options extends PipelineOptions {
+    @Default.String("package.OuterClass$InnerClass#method()")
+    String getString();
+
+    @JsonIgnore
+    Object getObject();
+  }
+
+  @Test
+  public void testAnnotationFormatter() throws Exception {
+    assertEquals(
+        "Default.String(value=package.OuterClass$InnerClass#method())",
+        ReflectHelpers.ANNOTATION_FORMATTER.apply(
+            Options.class.getMethod("getString").getAnnotations()[0]));
+
+    assertEquals(
+        "JsonIgnore(value=true)",
+        ReflectHelpers.ANNOTATION_FORMATTER.apply(
+            Options.class.getMethod("getObject").getAnnotations()[0]));
+  }
+
 }