You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/09/26 21:25:41 UTC

[1/2] incubator-beam git commit: Correctly type collections in PipelineOptions.

Repository: incubator-beam
Updated Branches:
  refs/heads/master 3fe3bc8eb -> 21c13d5f6


Correctly type collections in PipelineOptions.


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

Branch: refs/heads/master
Commit: 4e9987c423f8934f802b44f3e82a6200978bdc37
Parents: 3fe3bc8
Author: sammcveety <sa...@gmail.com>
Authored: Thu Aug 18 20:49:05 2016 -0400
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 26 14:25:08 2016 -0700

----------------------------------------------------------------------
 .../sdk/options/PipelineOptionsFactory.java     |  92 ++++++++-------
 .../sdk/options/PipelineOptionsFactoryTest.java | 111 +++++++++++++++++--
 2 files changed, 156 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4e9987c4/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 43927bc..9fc6c2c 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
@@ -28,8 +28,8 @@ import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.base.Strings;
-import com.google.common.collect.Collections2;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableListMultimap;
 import com.google.common.collect.ImmutableMap;
@@ -55,6 +55,7 @@ import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.Proxy;
+import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -843,13 +844,8 @@ public class PipelineOptionsFactory {
    * resolved.
    */
   private static List<PropertyDescriptor> getPropertyDescriptors(
-      Class<? extends PipelineOptions> beanClass)
+    Set<Method> methods, Class<? extends PipelineOptions> beanClass)
       throws IntrospectionException {
-    // The sorting is important to make this method stable.
-    SortedSet<Method> methods = Sets.newTreeSet(MethodComparator.INSTANCE);
-    methods.addAll(
-        Collections2.filter(Arrays.asList(beanClass.getMethods()), NOT_SYNTHETIC_PREDICATE));
-
     SortedMap<String, Method> propertyNamesToGetters = new TreeMap<>();
     for (Map.Entry<String, Method> entry :
         PipelineOptionsReflector.getPropertyNamesToGetters(methods).entries()) {
@@ -858,6 +854,7 @@ public class PipelineOptionsFactory {
 
     List<PropertyDescriptor> descriptors = Lists.newArrayList();
     List<TypeMismatch> mismatches = new ArrayList<>();
+    Set<String> usedDescriptors = Sets.newHashSet();
     /*
      * Add all the getter/setter pairs to the list of descriptors removing the getter once
      * it has been paired up.
@@ -874,9 +871,9 @@ public class PipelineOptionsFactory {
 
       // Validate that the getter and setter property types are the same.
       if (getterMethod != null) {
-        Class<?> getterPropertyType = getterMethod.getReturnType();
-        Class<?> setterPropertyType = method.getParameterTypes()[0];
-        if (getterPropertyType != setterPropertyType) {
+        Type getterPropertyType = getterMethod.getGenericReturnType();
+        Type setterPropertyType = method.getGenericParameterTypes()[0];
+        if (!getterPropertyType.equals(setterPropertyType)) {
           TypeMismatch mismatch = new TypeMismatch();
           mismatch.propertyName = propertyName;
           mismatch.getterPropertyType = getterPropertyType;
@@ -885,9 +882,14 @@ public class PipelineOptionsFactory {
           continue;
         }
       }
-
-      descriptors.add(new PropertyDescriptor(
-          propertyName, getterMethod, method));
+      // Properties can appear multiple times with subclasses, and we don't
+      // want to add a bad entry if we have already added a good one (with both
+      // getter and setter).
+      if (!usedDescriptors.contains(propertyName)) {
+        descriptors.add(new PropertyDescriptor(
+            propertyName, getterMethod, method));
+        usedDescriptors.add(propertyName);
+      }
     }
     throwForTypeMismatches(mismatches);
 
@@ -901,8 +903,8 @@ public class PipelineOptionsFactory {
 
   private static class TypeMismatch {
     private String propertyName;
-    private Class<?> getterPropertyType;
-    private Class<?> setterPropertyType;
+    private Type getterPropertyType;
+    private Type setterPropertyType;
   }
 
   private static void throwForTypeMismatches(List<TypeMismatch> mismatches) {
@@ -912,8 +914,8 @@ public class PipelineOptionsFactory {
           "Type mismatch between getter and setter methods for property [%s]. "
           + "Getter is of type [%s] whereas setter is of type [%s].",
           mismatch.propertyName,
-          mismatch.getterPropertyType.getName(),
-          mismatch.setterPropertyType.getName()));
+          mismatch.getterPropertyType,
+          mismatch.setterPropertyType));
     } else if (mismatches.size() > 1) {
       StringBuilder builder = new StringBuilder(
           "Type mismatches between getters and setters detected:");
@@ -921,8 +923,8 @@ public class PipelineOptionsFactory {
         builder.append(String.format(
             "%n  - Property [%s]: Getter is of type [%s] whereas setter is of type [%s].",
             mismatch.propertyName,
-            mismatch.getterPropertyType.getName(),
-            mismatch.setterPropertyType.getName()));
+            mismatch.getterPropertyType.toString(),
+            mismatch.setterPropertyType.toString()));
       }
       throw new IllegalArgumentException(builder.toString());
     }
@@ -977,14 +979,11 @@ public class PipelineOptionsFactory {
         methods.add(method);
       }
     }
-    // Ignore standard infrastructure methods on the generated class.
+    // Ignore methods on the base PipelineOptions interface.
     try {
-      methods.add(klass.getMethod("equals", Object.class));
-      methods.add(klass.getMethod("hashCode"));
-      methods.add(klass.getMethod("toString"));
-      methods.add(klass.getMethod("as", Class.class));
-      methods.add(klass.getMethod("cloneAs", Class.class));
-      methods.add(klass.getMethod("populateDisplayData", DisplayData.Builder.class));
+      methods.add(iface.getMethod("as", Class.class));
+      methods.add(iface.getMethod("cloneAs", Class.class));
+      methods.add(iface.getMethod("populateDisplayData", DisplayData.Builder.class));
     } catch (NoSuchMethodException | SecurityException e) {
       throw new RuntimeException(e);
     }
@@ -1017,7 +1016,7 @@ public class PipelineOptionsFactory {
 
     // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
     // that no setter has @JsonIgnore.
-    Iterable<Method> allInterfaceMethods =
+    SortedSet<Method> allInterfaceMethods =
         FluentIterable.from(
                 ReflectHelpers.getClosureOfMethodsOnInterfaces(
                     validatedPipelineOptionsInterfaces))
@@ -1030,7 +1029,7 @@ public class PipelineOptionsFactory {
       methodNameToAllMethodMap.put(method, method);
     }
 
-    List<PropertyDescriptor> descriptors = getPropertyDescriptors(klass);
+    List<PropertyDescriptor> descriptors = getPropertyDescriptors(allInterfaceMethods, iface);
 
     List<InconsistentlyIgnoredGetters> incompletelyIgnoredGetters = new ArrayList<>();
     List<IgnoredSetter> ignoredSetters = new ArrayList<>();
@@ -1104,13 +1103,25 @@ public class PipelineOptionsFactory {
       methods.add(propertyDescriptor.getWriteMethod());
     }
     throwForMissingBeanMethod(iface, missingBeanMethods);
+    final Set<String> knownMethods = Sets.newHashSet();
+    for (Method method : methods) {
+      knownMethods.add(method.getName());
+    }
 
     // Verify that no additional methods are on an interface that aren't a bean property.
+    // Because methods can have multiple declarations, we do a name-based comparison
+    // here to prevent false positives.
     SortedSet<Method> unknownMethods = new TreeSet<>(MethodComparator.INSTANCE);
     unknownMethods.addAll(
         Sets.filter(
-            Sets.difference(Sets.newHashSet(klass.getMethods()), methods),
-            NOT_SYNTHETIC_PREDICATE));
+            Sets.difference(Sets.newHashSet(iface.getMethods()), methods),
+            Predicates.and(NOT_SYNTHETIC_PREDICATE,
+                           new Predicate<Method>() {
+                             @Override
+                               public boolean apply(@Nonnull Method input) {
+                                 return !knownMethods.contains(input.getName());
+                             }
+                           })));
     checkArgument(unknownMethods.isEmpty(),
         "Methods %s on [%s] do not conform to being bean properties.",
         FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),
@@ -1402,9 +1413,8 @@ public class PipelineOptionsFactory {
                       klass, entry.getKey(), closestMatches));
           }
         }
-
         Method method = propertyNamesToGetters.get(entry.getKey());
-        // Only allow empty argument values for String, String Array, and Collection.
+        // Only allow empty argument values for String, String Array, and Collection<String>.
         Class<?> returnType = method.getReturnType();
         JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType());
         if ("runner".equals(entry.getKey())) {
@@ -1441,25 +1451,29 @@ public class PipelineOptionsFactory {
                 }
           }).toList();
 
-          if (returnType.isArray() && !returnType.getComponentType().equals(String.class)) {
+          if (returnType.isArray() && !returnType.getComponentType().equals(String.class)
+              || Collection.class.isAssignableFrom(returnType)) {
             for (String value : values) {
               checkArgument(!value.isEmpty(),
-                  "Empty argument value is only allowed for String, String Array, and Collection,"
-                  + " but received: " + returnType);
+                  "Empty argument value is only allowed for String, String Array, "
+                            + "and Collections of Strings, but received: %s",
+                            method.getGenericReturnType());
             }
           }
           convertedOptions.put(entry.getKey(), MAPPER.convertValue(values, type));
         } else if (SIMPLE_TYPES.contains(returnType) || returnType.isEnum()) {
           String value = Iterables.getOnlyElement(entry.getValue());
           checkArgument(returnType.equals(String.class) || !value.isEmpty(),
-              "Empty argument value is only allowed for String, String Array, and Collection,"
-               + " but received: " + returnType);
+               "Empty argument value is only allowed for String, String Array, "
+                        + "and Collections of Strings, but received: %s",
+                        method.getGenericReturnType());
           convertedOptions.put(entry.getKey(), MAPPER.convertValue(value, type));
         } else {
           String value = Iterables.getOnlyElement(entry.getValue());
           checkArgument(returnType.equals(String.class) || !value.isEmpty(),
-              "Empty argument value is only allowed for String, String Array, and Collection,"
-               + " but received: " + returnType);
+                "Empty argument value is only allowed for String, String Array, "
+                        + "and Collections of Strings, but received: %s",
+                        method.getGenericReturnType());
           try {
             convertedOptions.put(entry.getKey(), MAPPER.readValue(value, type));
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4e9987c4/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 70c8983..f26667f 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
@@ -217,8 +217,7 @@ public class PipelineOptionsFactoryTest {
     expectedException.expectMessage("Property [value]: Getter is of type "
         + "[boolean] whereas setter is of type [int].");
     expectedException.expectMessage("Property [other]: Getter is of type [long] "
-        + "whereas setter is of type [java.lang.String].");
-
+        + "whereas setter is of type [class java.lang.String].");
     PipelineOptionsFactory.as(MultiGetterSetterTypeMismatch.class);
   }
 
@@ -500,7 +499,8 @@ public class PipelineOptionsFactoryTest {
         "--byte="};
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "Empty argument value is only allowed for String, String Array, and Collection");
+        "Empty argument value is only allowed for String, String Array, and Collections"
+        + " of Strings");
     PipelineOptionsFactory.fromArgs(args).as(Primitives.class);
   }
 
@@ -708,7 +708,8 @@ public class PipelineOptionsFactoryTest {
   public void testEmptyInNonStringArrays() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "Empty argument value is only allowed for String, String Array, and Collection");
+        "Empty argument value is only allowed for String, String Array, and Collections"
+        + " of Strings");
 
     String[] args = new String[] {
         "--boolean=true",
@@ -722,7 +723,8 @@ public class PipelineOptionsFactoryTest {
   public void testEmptyInNonStringArraysWithCommaList() {
     expectedException.expect(IllegalArgumentException.class);
     expectedException.expectMessage(
-        "Empty argument value is only allowed for String, String Array, and Collection");
+        "Empty argument value is only allowed for String, String Array, and Collections"
+        + " of Strings");
 
     String[] args = new String[] {
         "--int=1,,9"};
@@ -749,16 +751,57 @@ public class PipelineOptionsFactoryTest {
   public static interface Lists extends PipelineOptions {
     List<String> getString();
     void setString(List<String> value);
+    List<Integer> getInteger();
+    void setInteger(List<Integer> value);
+    List getList();
+    void setList(List value);
   }
 
   @Test
-  public void testList() {
-    String[] args =
+  public void testListRawDefaultsToString() {
+    String[] manyArgs =
+        new String[] {"--list=stringValue1", "--list=stringValue2", "--list=stringValue3"};
+
+    Lists options = PipelineOptionsFactory.fromArgs(manyArgs).as(Lists.class);
+    assertEquals(ImmutableList.of("stringValue1", "stringValue2", "stringValue3"),
+        options.getList());
+  }
+
+  @Test
+  public void testListString() {
+    String[] manyArgs =
         new String[] {"--string=stringValue1", "--string=stringValue2", "--string=stringValue3"};
+    String[] oneArg = new String[] {"--string=stringValue1"};
 
-    Lists options = PipelineOptionsFactory.fromArgs(args).as(Lists.class);
+    Lists options = PipelineOptionsFactory.fromArgs(manyArgs).as(Lists.class);
     assertEquals(ImmutableList.of("stringValue1", "stringValue2", "stringValue3"),
         options.getString());
+
+    options = PipelineOptionsFactory.fromArgs(oneArg).as(Lists.class);
+    assertEquals(ImmutableList.of("stringValue1"), options.getString());
+  }
+
+  @Test
+  public void testListInt() {
+    String[] manyArgs =
+        new String[] {"--integer=1", "--integer=2", "--integer=3"};
+    String[] manyArgsShort =
+        new String[] {"--integer=1,2,3"};
+    String[] oneArg = new String[] {"--integer=1"};
+    String[] missingArg = new String[] {"--integer="};
+
+    Lists options = PipelineOptionsFactory.fromArgs(manyArgs).as(Lists.class);
+    assertEquals(ImmutableList.of(1, 2, 3), options.getInteger());
+    options = PipelineOptionsFactory.fromArgs(manyArgsShort).as(Lists.class);
+    assertEquals(ImmutableList.of(1, 2, 3), options.getInteger());
+    options = PipelineOptionsFactory.fromArgs(oneArg).as(Lists.class);
+    assertEquals(ImmutableList.of(1), options.getInteger());
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+      "Empty argument value is only allowed for String, String Array, and Collections of Strings,"
+      + " but received: java.util.List<java.lang.Integer>");
+    options = PipelineOptionsFactory.fromArgs(missingArg).as(Lists.class);
   }
 
   @Test
@@ -806,6 +849,58 @@ public class PipelineOptionsFactoryTest {
     expectedLogs.verifyWarn("Strict parsing is disabled, ignoring option");
   }
 
+  /** A test interface containing all supported List return types. */
+  public static interface Maps extends PipelineOptions {
+    Map<Integer, Integer> getMap();
+    void setMap(Map<Integer, Integer> value);
+
+    Map<Integer, Map<Integer, Integer>> getNestedMap();
+    void setNestedMap(Map<Integer, Map<Integer, Integer>> value);
+  }
+
+  @Test
+  public void testMapIntInt() {
+    String[] manyArgsShort =
+        new String[] {"--map={\"1\":1,\"2\":2}"};
+    String[] oneArg = new String[] {"--map={\"1\":1}"};
+    String[] missingArg = new String[] {"--map="};
+
+    Maps options = PipelineOptionsFactory.fromArgs(manyArgsShort).as(Maps.class);
+    assertEquals(ImmutableMap.of(1, 1, 2, 2), options.getMap());
+    options = PipelineOptionsFactory.fromArgs(oneArg).as(Maps.class);
+    assertEquals(ImmutableMap.of(1, 1), options.getMap());
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+      "Empty argument value is only allowed for String, String Array, and "
+      + "Collections of Strings, but received: java.util.Map<java.lang.Integer, "
+      + "java.lang.Integer>");
+    options = PipelineOptionsFactory.fromArgs(missingArg).as(Maps.class);
+  }
+
+  @Test
+  public void testNestedMap() {
+    String[] manyArgsShort =
+        new String[] {"--nestedMap={\"1\":{\"1\":1},\"2\":{\"2\":2}}"};
+    String[] oneArg = new String[] {"--nestedMap={\"1\":{\"1\":1}}"};
+    String[] missingArg = new String[] {"--nestedMap="};
+
+    Maps options = PipelineOptionsFactory.fromArgs(manyArgsShort).as(Maps.class);
+    assertEquals(ImmutableMap.of(1, ImmutableMap.of(1, 1),
+                                 2, ImmutableMap.of(2, 2)),
+                 options.getNestedMap());
+    options = PipelineOptionsFactory.fromArgs(oneArg).as(Maps.class);
+    assertEquals(ImmutableMap.of(1, ImmutableMap.of(1, 1)),
+                 options.getNestedMap());
+
+    expectedException.expect(IllegalArgumentException.class);
+    expectedException.expectMessage(
+      "Empty argument value is only allowed for String, String Array, and Collections of "
+      + "Strings, but received: java.util.Map<java.lang.Integer, "
+      + "java.util.Map<java.lang.Integer, java.lang.Integer>>");
+    options = PipelineOptionsFactory.fromArgs(missingArg).as(Maps.class);
+  }
+
   @Test
   public void testSettingRunner() {
     String[] args = new String[] {"--runner=" + RegisteredTestRunner.class.getSimpleName()};


[2/2] incubator-beam git commit: Closes #852

Posted by dh...@apache.org.
Closes #852


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

Branch: refs/heads/master
Commit: 21c13d5f6f6913bda2dcfb52157f2e36aa86bf8b
Parents: 3fe3bc8 4e9987c
Author: Dan Halperin <dh...@google.com>
Authored: Mon Sep 26 14:25:31 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Sep 26 14:25:31 2016 -0700

----------------------------------------------------------------------
 .../sdk/options/PipelineOptionsFactory.java     |  92 ++++++++-------
 .../sdk/options/PipelineOptionsFactoryTest.java | 111 +++++++++++++++++--
 2 files changed, 156 insertions(+), 47 deletions(-)
----------------------------------------------------------------------