You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2016/03/20 01:06:33 UTC

[1/2] incubator-beam git commit: Filter Synthetic Methods in PipelineOptionsFactory

Repository: incubator-beam
Updated Branches:
  refs/heads/master a461e006a -> f7aaee2ea


Filter Synthetic Methods in PipelineOptionsFactory


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

Branch: refs/heads/master
Commit: 91de07245ad2470265d1d0d4a23873cc7680b495
Parents: a461e00
Author: Thomas Groh <tg...@google.com>
Authored: Mon Mar 7 13:47:30 2016 -0800
Committer: Luke Cwik <lc...@google.com>
Committed: Sat Mar 19 15:11:01 2016 -0700

----------------------------------------------------------------------
 .../dataflow/sdk/options/PipelineOptions.java   |  3 +-
 .../sdk/options/PipelineOptionsFactory.java     | 41 ++++++---
 .../PipelineOptionsFactoryJava8Test.java        | 90 ++++++++++++++++++++
 3 files changed, 122 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/91de0724/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
index 923033d..8ff1fa9 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java
@@ -137,7 +137,8 @@ import javax.annotation.concurrent.ThreadSafe;
  *
  * <p>{@link Default @Default} represents a set of annotations that can be used to annotate getter
  * properties on {@link PipelineOptions} with information representing the default value to be
- * returned if no value is specified.
+ * returned if no value is specified. Any default implementation (using the {@code default} keyword)
+ * is ignored.
  *
  * <p>{@link Hidden @Hidden} hides an option from being listed when {@code --help}
  * is invoked via {@link PipelineOptionsFactory#fromArgs(String[])}.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/91de0724/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
index 48cff6d..4781d1c 100644
--- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
+++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java
@@ -445,13 +445,21 @@ public class PipelineOptionsFactory {
   private static final Map<String, Class<? extends PipelineRunner<?>>> SUPPORTED_PIPELINE_RUNNERS;
 
   /** Classes that are used as the boundary in the stack trace to find the callers class name. */
-  private static final Set<String> PIPELINE_OPTIONS_FACTORY_CLASSES = ImmutableSet.of(
-      PipelineOptionsFactory.class.getName(),
-      Builder.class.getName());
+  private static final Set<String> PIPELINE_OPTIONS_FACTORY_CLASSES =
+      ImmutableSet.of(PipelineOptionsFactory.class.getName(), Builder.class.getName());
 
   /** Methods that are ignored when validating the proxy class. */
   private static final Set<Method> IGNORED_METHODS;
 
+  /** A predicate that checks if a method is synthetic via {@link Method#isSynthetic()}. */
+  private static final Predicate<Method> NOT_SYNTHETIC_PREDICATE =
+      new Predicate<Method>() {
+        @Override
+        public boolean apply(Method input) {
+          return !input.isSynthetic();
+        }
+      };
+
   /** The set of options that have been registered and visible to the user. */
   private static final Set<Class<? extends PipelineOptions>> REGISTERED_OPTIONS =
       Sets.newConcurrentHashSet();
@@ -664,7 +672,9 @@ public class PipelineOptionsFactory {
     Preconditions.checkNotNull(iface);
     validateWellFormed(iface, REGISTERED_OPTIONS);
 
-    Iterable<Method> methods = ReflectHelpers.getClosureOfMethodsOnInterface(iface);
+    Iterable<Method> methods =
+        Iterables.filter(
+            ReflectHelpers.getClosureOfMethodsOnInterface(iface), NOT_SYNTHETIC_PREDICATE);
     ListMultimap<Class<?>, Method> ifaceToMethods = ArrayListMultimap.create();
     for (Method method : methods) {
       // Process only methods that are not marked as hidden.
@@ -878,7 +888,8 @@ public class PipelineOptionsFactory {
       throws IntrospectionException {
     // The sorting is important to make this method stable.
     SortedSet<Method> methods = Sets.newTreeSet(MethodComparator.INSTANCE);
-    methods.addAll(Arrays.asList(beanClass.getMethods()));
+    methods.addAll(
+        Collections2.filter(Arrays.asList(beanClass.getMethods()), NOT_SYNTHETIC_PREDICATE));
     SortedMap<String, Method> propertyNamesToGetters = getPropertyNamesToGetters(methods);
     List<PropertyDescriptor> descriptors = Lists.newArrayList();
 
@@ -1019,8 +1030,9 @@ public class PipelineOptionsFactory {
       Class<?> klass) throws IntrospectionException {
     Set<Method> methods = Sets.newHashSet(IGNORED_METHODS);
     // Ignore static methods, "equals", "hashCode", "toString" and "as" on the generated class.
+    // Ignore synthetic methods
     for (Method method : klass.getMethods()) {
-      if (Modifier.isStatic(method.getModifiers())) {
+      if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) {
         methods.add(method);
       }
     }
@@ -1037,6 +1049,7 @@ public class PipelineOptionsFactory {
     // Verify that there are no methods with the same name with two different return types.
     Iterable<Method> interfaceMethods = FluentIterable
         .from(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
+        .filter(NOT_SYNTHETIC_PREDICATE)
         .toSortedSet(MethodComparator.INSTANCE);
     SortedSetMultimap<Method, Method> methodNameToMethodMap =
         TreeMultimap.create(MethodNameComparator.INSTANCE, MethodComparator.INSTANCE);
@@ -1061,10 +1074,13 @@ public class PipelineOptionsFactory {
 
     // Verify that there is no getter with a mixed @JsonIgnore annotation and verify
     // that no setter has @JsonIgnore.
-    Iterable<Method> allInterfaceMethods = FluentIterable
-        .from(ReflectHelpers.getClosureOfMethodsOnInterfaces(validatedPipelineOptionsInterfaces))
-        .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
-        .toSortedSet(MethodComparator.INSTANCE);
+    Iterable<Method> allInterfaceMethods =
+        FluentIterable.from(
+                ReflectHelpers.getClosureOfMethodsOnInterfaces(
+                    validatedPipelineOptionsInterfaces))
+            .append(ReflectHelpers.getClosureOfMethodsOnInterface(iface))
+            .filter(NOT_SYNTHETIC_PREDICATE)
+            .toSortedSet(MethodComparator.INSTANCE);
     SortedSetMultimap<Method, Method> methodNameToAllMethodMap =
         TreeMultimap.create(MethodNameComparator.INSTANCE, MethodComparator.INSTANCE);
     for (Method method : allInterfaceMethods) {
@@ -1148,7 +1164,10 @@ public class PipelineOptionsFactory {
 
     // Verify that no additional methods are on an interface that aren't a bean property.
     SortedSet<Method> unknownMethods = new TreeSet<>(MethodComparator.INSTANCE);
-    unknownMethods.addAll(Sets.difference(Sets.newHashSet(klass.getMethods()), methods));
+    unknownMethods.addAll(
+        Sets.filter(
+            Sets.difference(Sets.newHashSet(klass.getMethods()), methods),
+            NOT_SYNTHETIC_PREDICATE));
     Preconditions.checkArgument(unknownMethods.isEmpty(),
         "Methods %s on [%s] do not conform to being bean properties.",
         FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/91de0724/sdk/src/test/java8/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryJava8Test.java
----------------------------------------------------------------------
diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryJava8Test.java
new file mode 100644
index 0000000..b7e1467
--- /dev/null
+++ b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactoryJava8Test.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright (C) 2016 Google Inc.
+ *
+ * Licensed 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
+ *
+ * http://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 com.google.cloud.dataflow.sdk.options;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Java 8 tests for {@link PipelineOptionsFactory}.
+ */
+@RunWith(JUnit4.class)
+public class PipelineOptionsFactoryJava8Test {
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  private static interface OptionsWithDefaultMethod extends PipelineOptions {
+    default Number getValue() {
+      return 1024;
+    }
+
+    void setValue(Number value);
+  }
+
+  @Test
+  public void testDefaultMethodIgnoresDefaultImplementation() {
+    OptionsWithDefaultMethod optsWithDefault =
+        PipelineOptionsFactory.as(OptionsWithDefaultMethod.class);
+    assertThat(optsWithDefault.getValue(), nullValue());
+
+    optsWithDefault.setValue(12.25);
+    assertThat(optsWithDefault.getValue(), equalTo(Double.valueOf(12.25)));
+  }
+
+  private static interface ExtendedOptionsWithDefault extends OptionsWithDefaultMethod {}
+
+  @Test
+  public void testDefaultMethodInExtendedClassIgnoresDefaultImplementation() {
+    OptionsWithDefaultMethod extendedOptsWithDefault =
+        PipelineOptionsFactory.as(ExtendedOptionsWithDefault.class);
+    assertThat(extendedOptsWithDefault.getValue(), nullValue());
+
+    extendedOptsWithDefault.setValue(Double.NEGATIVE_INFINITY);
+    assertThat(extendedOptsWithDefault.getValue(), equalTo(Double.NEGATIVE_INFINITY));
+  }
+
+  private static interface Options extends PipelineOptions {
+    Number getValue();
+
+    void setValue(Number value);
+  }
+
+  private static interface SubtypeReturingOptions extends Options {
+    @Override
+    Integer getValue();
+    void setValue(Integer value);
+  }
+
+  @Test
+  public void testReturnTypeConflictThrows() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage(
+        "Method [getValue] has multiple definitions [public abstract java.lang.Integer "
+            + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryJava8Test$"
+            + "SubtypeReturingOptions.getValue(), public abstract java.lang.Number "
+            + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryJava8Test$Options"
+            + ".getValue()] with different return types for ["
+            + "com.google.cloud.dataflow.sdk.options.PipelineOptionsFactoryJava8Test$"
+            + "SubtypeReturingOptions].");
+    PipelineOptionsFactory.as(SubtypeReturingOptions.class);
+  }
+}


[2/2] incubator-beam git commit: This closes #41

Posted by lc...@apache.org.
This closes #41


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

Branch: refs/heads/master
Commit: f7aaee2ea759c0687588e62e5603ad50332d18ea
Parents: a461e00 91de072
Author: Luke Cwik <lc...@google.com>
Authored: Sat Mar 19 15:12:26 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Sat Mar 19 15:12:26 2016 -0700

----------------------------------------------------------------------
 .../dataflow/sdk/options/PipelineOptions.java   |  3 +-
 .../sdk/options/PipelineOptionsFactory.java     | 41 ++++++---
 .../PipelineOptionsFactoryJava8Test.java        | 90 ++++++++++++++++++++
 3 files changed, 122 insertions(+), 12 deletions(-)
----------------------------------------------------------------------