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/10/25 16:28:30 UTC

[12/50] incubator-beam git commit: [BEAM-790] Refactor PipelineOptionsFactory validation into separate methods.

[BEAM-790] Refactor PipelineOptionsFactory validation into separate methods.


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

Branch: refs/heads/apex-runner
Commit: b809883813a992167bfa4ad59759fb992c0694d2
Parents: ea8583c
Author: Pei He <pe...@google.com>
Authored: Thu Oct 20 16:32:45 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Fri Oct 21 15:43:00 2016 -0700

----------------------------------------------------------------------
 .../sdk/options/PipelineOptionsFactory.java     | 151 +++++++++++++------
 1 file changed, 105 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b8098838/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 7206b11..eefe8c7 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
@@ -958,7 +958,7 @@ public class PipelineOptionsFactory {
   /**
    * Validates that a given class conforms to the following properties:
    * <ul>
-   *   <li>Any property with the same name must have the same return type for all derived
+   *   <li>Any method with the same name must have the same return type for all derived
    *       interfaces of {@link PipelineOptions}.
    *   <li>Every bean property of any interface derived from {@link PipelineOptions} must have a
    *       getter and setter method.
@@ -979,22 +979,37 @@ public class PipelineOptionsFactory {
   private static List<PropertyDescriptor> validateClass(Class<? extends PipelineOptions> iface,
       Set<Class<? extends PipelineOptions>> validatedPipelineOptionsInterfaces,
       Class<? extends PipelineOptions> klass) throws IntrospectionException {
-    Set<Method> methods = Sets.newHashSet(IGNORED_METHODS);
-    // Ignore synthetic methods
-    for (Method method : klass.getMethods()) {
-      if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) {
-        methods.add(method);
-      }
-    }
-    // Ignore methods on the base PipelineOptions interface.
-    try {
-      methods.add(iface.getMethod("as", Class.class));
-      methods.add(iface.getMethod("populateDisplayData", DisplayData.Builder.class));
-    } catch (NoSuchMethodException | SecurityException e) {
-      throw new RuntimeException(e);
-    }
-
     // Verify that there are no methods with the same name with two different return types.
+    validateReturnType(iface);
+
+    SortedSet<Method> allInterfaceMethods = FluentIterable
+        .from(ReflectHelpers.getClosureOfMethodsOnInterfaces(
+            validatedPipelineOptionsInterfaces))
+        .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
+        .filter(NOT_SYNTHETIC_PREDICATE)
+        .toSortedSet(MethodComparator.INSTANCE);
+
+    List<PropertyDescriptor> descriptors = getPropertyDescriptors(allInterfaceMethods, iface);
+
+    // Verify that all method annotations are valid.
+    validateMethodAnnotations(allInterfaceMethods, descriptors);
+
+    // Verify that each property has a matching read and write method.
+    validateGettersSetters(iface, descriptors);
+
+    // Verify all methods are bean methods or known methods.
+    validateMethodsAreEitherBeanMethodOrKnownMethod(iface, klass, descriptors);
+
+    return descriptors;
+  }
+
+  /**
+   * Validates that any method with the same name must have the same return type for all derived
+   * interfaces of {@link PipelineOptions}.
+   *
+   * @param iface The interface to validate.
+   */
+  private static void validateReturnType(Class<? extends PipelineOptions> iface) {
     Iterable<Method> interfaceMethods = FluentIterable
         .from(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
         .filter(NOT_SYNTHETIC_PREDICATE)
@@ -1019,24 +1034,29 @@ public class PipelineOptionsFactory {
       }
     }
     throwForMultipleDefinitions(iface, multipleDefinitions);
+  }
 
-    // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
-    // that no setter has @JsonIgnore.
-    SortedSet<Method> allInterfaceMethods =
-        FluentIterable.from(
-                ReflectHelpers.getClosureOfMethodsOnInterfaces(
-                    validatedPipelineOptionsInterfaces))
-            .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
-            .filter(NOT_SYNTHETIC_PREDICATE)
-            .toSortedSet(MethodComparator.INSTANCE);
+  /**
+   * Validates that a given class conforms to the following properties:
+   * <ul>
+   *   <li>Only getters may be annotated with {@link JsonIgnore @JsonIgnore}.
+   *   <li>If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for
+   *       this property must be annotated with {@link JsonIgnore @JsonIgnore}.
+   * </ul>
+   *
+   * @param allInterfaceMethods All interface methods that derive from {@link PipelineOptions}.
+   * @param descriptors The list of {@link PropertyDescriptor}s representing all valid bean
+   * properties of {@code iface}.
+   */
+  private static void validateMethodAnnotations(
+      SortedSet<Method> allInterfaceMethods,
+      List<PropertyDescriptor> descriptors) {
     SortedSetMultimap<Method, Method> methodNameToAllMethodMap =
         TreeMultimap.create(MethodNameComparator.INSTANCE, MethodComparator.INSTANCE);
     for (Method method : allInterfaceMethods) {
       methodNameToAllMethodMap.put(method, method);
     }
 
-    List<PropertyDescriptor> descriptors = getPropertyDescriptors(allInterfaceMethods, iface);
-
     List<InconsistentlyIgnoredGetters> incompletelyIgnoredGetters = new ArrayList<>();
     List<IgnoredSetter> ignoredSetters = new ArrayList<>();
 
@@ -1047,6 +1067,8 @@ public class PipelineOptionsFactory {
           || IGNORED_METHODS.contains(descriptor.getWriteMethod())) {
         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);
 
@@ -1073,8 +1095,8 @@ public class PipelineOptionsFactory {
               JsonIgnorePredicate.INSTANCE);
 
       Iterable<String> settersWithJsonIgnoreClassNames = FluentIterable.from(settersWithJsonIgnore)
-              .transform(MethodToDeclaringClassFunction.INSTANCE)
-              .transform(ReflectHelpers.CLASS_NAME);
+          .transform(MethodToDeclaringClassFunction.INSTANCE)
+          .transform(ReflectHelpers.CLASS_NAME);
 
       if (!settersWithJsonIgnore.isEmpty()) {
         IgnoredSetter ignored = new IgnoredSetter();
@@ -1085,12 +1107,22 @@ public class PipelineOptionsFactory {
     }
     throwForGettersWithInconsistentJsonIgnore(incompletelyIgnoredGetters);
     throwForSettersWithJsonIgnore(ignoredSetters);
+  }
 
+  /**
+   * Validates that every bean property of the given interface must have both a getter and setter.
+   *
+   * @param iface The interface to validate.
+   * @param descriptors The list of {@link PropertyDescriptor}s representing all valid bean
+   * properties of {@code iface}.
+   */
+  private static void validateGettersSetters(
+      Class<? extends PipelineOptions> iface,
+      List<PropertyDescriptor> descriptors) {
     List<MissingBeanMethod> missingBeanMethods = new ArrayList<>();
-    // Verify that each property has a matching read and write method.
     for (PropertyDescriptor propertyDescriptor : descriptors) {
       if (!(IGNORED_METHODS.contains(propertyDescriptor.getWriteMethod())
-        || propertyDescriptor.getReadMethod() != null)) {
+          || propertyDescriptor.getReadMethod() != null)) {
         MissingBeanMethod method = new MissingBeanMethod();
         method.property = propertyDescriptor;
         method.methodType = "getter";
@@ -1098,20 +1130,49 @@ public class PipelineOptionsFactory {
         continue;
       }
       if (!(IGNORED_METHODS.contains(propertyDescriptor.getReadMethod())
-              || propertyDescriptor.getWriteMethod() != null)) {
+          || propertyDescriptor.getWriteMethod() != null)) {
         MissingBeanMethod method = new MissingBeanMethod();
         method.property = propertyDescriptor;
         method.methodType = "setter";
         missingBeanMethods.add(method);
         continue;
       }
-      methods.add(propertyDescriptor.getReadMethod());
-      methods.add(propertyDescriptor.getWriteMethod());
     }
     throwForMissingBeanMethod(iface, missingBeanMethods);
-    final Set<String> knownMethods = Sets.newHashSet();
-    for (Method method : methods) {
-      knownMethods.add(method.getName());
+  }
+
+  /**
+   * Validates that every non-static or synthetic method is either a known method such as
+   * {@link PipelineOptions#as} or a bean property.
+   *
+   * @param iface The interface to validate.
+   * @param klass The proxy class representing the interface.
+   */
+  private static void validateMethodsAreEitherBeanMethodOrKnownMethod(
+      Class<? extends PipelineOptions> iface,
+      Class<? extends PipelineOptions> klass,
+      List<PropertyDescriptor> descriptors) {
+    Set<Method> knownMethods = Sets.newHashSet(IGNORED_METHODS);
+    // Ignore synthetic methods
+    for (Method method : klass.getMethods()) {
+      if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) {
+        knownMethods.add(method);
+      }
+    }
+    // Ignore methods on the base PipelineOptions interface.
+    try {
+      knownMethods.add(iface.getMethod("as", Class.class));
+      knownMethods.add(iface.getMethod("populateDisplayData", DisplayData.Builder.class));
+    } catch (NoSuchMethodException | SecurityException e) {
+      throw new RuntimeException(e);
+    }
+    for (PropertyDescriptor descriptor : descriptors) {
+      knownMethods.add(descriptor.getReadMethod());
+      knownMethods.add(descriptor.getWriteMethod());
+    }
+    final Set<String> knownMethodsNames = Sets.newHashSet();
+    for (Method method : knownMethods) {
+      knownMethodsNames.add(method.getName());
     }
 
     // Verify that no additional methods are on an interface that aren't a bean property.
@@ -1120,20 +1181,18 @@ public class PipelineOptionsFactory {
     SortedSet<Method> unknownMethods = new TreeSet<>(MethodComparator.INSTANCE);
     unknownMethods.addAll(
         Sets.filter(
-            Sets.difference(Sets.newHashSet(iface.getMethods()), methods),
+            Sets.difference(Sets.newHashSet(iface.getMethods()), knownMethods),
             Predicates.and(NOT_SYNTHETIC_PREDICATE,
-                           new Predicate<Method>() {
-                             @Override
-                               public boolean apply(@Nonnull Method input) {
-                                 return !knownMethods.contains(input.getName());
-                             }
-                           })));
+                new Predicate<Method>() {
+                  @Override
+                  public boolean apply(@Nonnull Method input) {
+                    return !knownMethodsNames.contains(input.getName());
+                  }
+                })));
     checkArgument(unknownMethods.isEmpty(),
         "Methods %s on [%s] do not conform to being bean properties.",
         FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),
         iface.getName());
-
-    return descriptors;
   }
 
   private static class MultipleDefinitions {